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

#include "db/db_impl.h"

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

I
Igor Canadi 已提交
16
#include <inttypes.h>
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
#include "rocksdb/compaction_filter.h"
#include "rocksdb/db.h"
I
Islam AbdelRahman 已提交
57
#include "rocksdb/delete_scheduler.h"
58 59
#include "rocksdb/env.h"
#include "rocksdb/merge_operator.h"
I
Igor Canadi 已提交
60
#include "rocksdb/version.h"
61 62
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
S
Siying Dong 已提交
63
#include "rocksdb/table.h"
J
jorlow@chromium.org 已提交
64
#include "table/block.h"
65
#include "table/block_based_table_factory.h"
J
jorlow@chromium.org 已提交
66
#include "table/merger.h"
K
kailiu 已提交
67
#include "table/table_builder.h"
J
jorlow@chromium.org 已提交
68
#include "table/two_level_iterator.h"
69
#include "util/auto_roll_logger.h"
K
kailiu 已提交
70
#include "util/autovector.h"
71
#include "util/build_version.h"
J
jorlow@chromium.org 已提交
72
#include "util/coding.h"
I
Igor Canadi 已提交
73
#include "util/compression.h"
74
#include "util/crc32c.h"
75
#include "util/db_info_dumper.h"
76
#include "util/file_reader_writer.h"
77
#include "util/file_util.h"
I
Igor Canadi 已提交
78
#include "util/hash_skiplist_rep.h"
79
#include "util/hash_linklist_rep.h"
J
jorlow@chromium.org 已提交
80
#include "util/logging.h"
H
Haobo Xu 已提交
81
#include "util/log_buffer.h"
J
jorlow@chromium.org 已提交
82
#include "util/mutexlock.h"
83
#include "util/perf_context_imp.h"
84
#include "util/iostats_context_imp.h"
85
#include "util/stop_watch.h"
86
#include "util/sync_point.h"
87
#include "util/string_util.h"
88 89
#include "util/thread_status_updater.h"
#include "util/thread_status_util.h"
V
Venkatesh Radhakrishnan 已提交
90
#include "util/xfunc.h"
J
jorlow@chromium.org 已提交
91

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

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

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

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

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

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

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

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

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

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

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

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

J
jorlow@chromium.org 已提交
159 160 161
  return result;
}

162 163
namespace {

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

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

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

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

200
void DumpSupportInfo(Logger* logger) {
I
Igor Canadi 已提交
201 202 203 204 205 206 207 208
  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());
209 210
  Log(InfoLogLevel::INFO_LEVEL, logger, "Fast CRC32 supported: %d",
      crc32c::IsFastCrc32Supported());
I
Igor Canadi 已提交
211 212
}

213
}  // namespace
214

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

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

265
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
266 267
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
I
Igor Canadi 已提交
268 269
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
270

I
Igor Canadi 已提交
271
  DumpRocksDBBuildVersion(db_options_.info_log.get());
272 273
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
274
  DumpSupportInfo(db_options_.info_log.get());
275

276
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
277 278
}

279
// Will lock the mutex_,  will wait for completion if wait is true
280
void DBImpl::CancelAllBackgroundWork(bool wait) {
281
  InstrumentedMutexLock l(&mutex_);
282
  shutting_down_.store(true, std::memory_order_release);
283
  bg_cv_.SignalAll();
284 285 286 287
  if (!wait) {
    return;
  }
  // Wait for background work to finish
288
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
289 290
    bg_cv_.Wait();
  }
291 292
}

J
jorlow@chromium.org 已提交
293
DBImpl::~DBImpl() {
294
  mutex_.Lock();
295

296
  if (!shutting_down_.load(std::memory_order_acquire) && flush_on_destroy_) {
297
    for (auto cfd : *versions_->GetColumnFamilySet()) {
298
      if (!cfd->IsDropped() && !cfd->mem()->IsEmpty()) {
299 300 301 302
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
303
        cfd->Unref();
304 305
      }
    }
306
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
307
  }
308 309 310
  mutex_.Unlock();
  // CancelAllBackgroundWork called with false means we just set the shutdown
  // marker. After this we do a variant of the waiting and unschedule work
311 312
  // (to consider: moving all the waiting into CancelAllBackgroundWork(true))
  CancelAllBackgroundWork(false);
313 314 315 316 317 318 319
  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
320
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
321 322
    bg_cv_.Wait();
  }
323
  EraseThreadStatusDbInfo();
I
Igor Canadi 已提交
324 325
  flush_scheduler_.Clear();

326 327 328 329 330 331 332 333 334 335 336 337 338
  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 已提交
339 340 341 342 343
  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();
344 345
  }

I
Igor Canadi 已提交
346 347 348 349 350 351 352 353 354 355
  // 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_) {
356
    JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
357
    FindObsoleteFiles(&job_context, true);
358 359

    mutex_.Unlock();
I
Igor Canadi 已提交
360
    // manifest number starting from 2
I
Igor Canadi 已提交
361 362 363
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
364
    }
I
Igor Canadi 已提交
365
    job_context.Clean();
366
    mutex_.Lock();
367 368
  }

369 370 371
  for (auto l : logs_to_free_) {
    delete l;
  }
372 373 374
  for (auto& log : logs_) {
    log.ClearWriter();
  }
375
  logs_.clear();
376

377
  // versions need to be destroyed before table_cache since it can hold
378 379
  // references to table_cache.
  versions_.reset();
380
  mutex_.Unlock();
I
Igor Canadi 已提交
381 382 383
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
384

385
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
386 387 388
}

Status DBImpl::NewDB() {
389
  VersionEdit new_db;
390
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
391 392 393
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

394 395
  Status s;

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

void DBImpl::MaybeIgnoreError(Status* s) const {
427
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
428 429
    // No change needed
  } else {
430 431
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
432 433 434 435
    *s = Status::OK();
  }
}

436
const Status DBImpl::CreateArchivalDirectory() {
437 438
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
439 440 441 442 443
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

444
void DBImpl::PrintStatistics() {
445
  auto dbstats = db_options_.statistics.get();
446
  if (dbstats) {
Y
Yueh-Hsuan Chiang 已提交
447
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
M
Mark Callaghan 已提交
448
        "STATISTICS:\n %s",
449
        dbstats->ToString().c_str());
450 451 452
  }
}

453
void DBImpl::MaybeDumpStats() {
454
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
455 456 457 458

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
459
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
460 461 462 463 464 465
      <= 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;
466

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

492
    PrintStatistics();
493 494 495
  }
}

I
Igor Canadi 已提交
496
// * Returns the list of live files in 'sst_live'
I
Igor Canadi 已提交
497
// If it's doing full scan:
I
Igor Canadi 已提交
498 499
// * Returns the list of all files in the filesystem in
// 'full_scan_candidate_files'.
I
Igor Canadi 已提交
500
// Otherwise, gets obsolete files from VersionSet.
I
Igor Canadi 已提交
501 502
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
503
//  db_options_.delete_obsolete_files_period_micros
I
Igor Canadi 已提交
504
// force = true -- force the full scan
I
Igor Canadi 已提交
505
void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
I
Igor Canadi 已提交
506
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
507 508
  mutex_.AssertHeld();

509
  // if deletion is disabled, do nothing
510
  if (disable_delete_obsolete_files_ > 0) {
511 512 513
    return;
  }

514 515 516 517 518
  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;
519
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
520 521 522
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
I
Igor Canadi 已提交
523
    if (delete_obsolete_files_next_run_ < now_micros) {
524
      doing_the_full_scan = true;
I
Igor Canadi 已提交
525 526
      delete_obsolete_files_next_run_ =
          now_micros + db_options_.delete_obsolete_files_period_micros;
527 528 529
    }
  }

I
Igor Canadi 已提交
530 531 532 533 534 535 536 537 538
  // 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();
  }

539 540
  // Get obsolete files.  This function will also update the list of
  // pending files in VersionSet().
I
Igor Canadi 已提交
541 542
  versions_->GetObsoleteFiles(&job_context->sst_delete_files,
                              job_context->min_pending_output);
I
Igor Canadi 已提交
543

544
  if (!alive_log_files_.empty()) {
545
    uint64_t min_log_number = versions_->MinLogNumber();
546 547 548 549 550 551 552 553 554 555
    // find newly obsoleted log files
    while (alive_log_files_.begin()->number < min_log_number) {
      auto& earliest = *alive_log_files_.begin();
      job_context->log_delete_files.push_back(earliest.number);
      total_log_size_ -= earliest.size;
      alive_log_files_.pop_front();
      // Current log should always stay alive since it can't have
      // number < MinLogNumber().
      assert(alive_log_files_.size());
    }
556 557 558 559 560 561 562
    while (!logs_.empty() && logs_.front().number < min_log_number) {
      auto& log = logs_.front();
      if (log.getting_synced) {
        log_sync_cv_.Wait();
        // logs_ could have changed while we were waiting.
        continue;
      }
563
      logs_to_free_.push_back(log.ReleaseWriter());
564 565 566 567
      logs_.pop_front();
    }
    // Current log cannot be obsolete.
    assert(!logs_.empty());
568 569 570
  }

  // We're just cleaning up for DB::Write().
Y
Yueh-Hsuan Chiang 已提交
571
  assert(job_context->logs_to_free.empty());
572 573 574
  job_context->logs_to_free = logs_to_free_;
  logs_to_free_.clear();

I
Igor Canadi 已提交
575
  // store the current filenum, lognum, etc
576
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
577
  job_context->pending_manifest_file_number =
578
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
579
  job_context->log_number = versions_->MinLogNumber();
580
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
581

I
Igor Canadi 已提交
582
  versions_->AddLiveFiles(&job_context->sst_live);
583
  if (doing_the_full_scan) {
I
Igor Canadi 已提交
584 585
    for (uint32_t path_id = 0; path_id < db_options_.db_paths.size();
         path_id++) {
586 587 588
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
589
      env_->GetChildren(db_options_.db_paths[path_id].path,
590
                        &files);  // Ignore errors
591
      for (std::string file : files) {
I
Igor Canadi 已提交
592
        // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
I
Igor Canadi 已提交
593 594
        job_context->full_scan_candidate_files.emplace_back("/" + file,
                                                            path_id);
595 596
      }
    }
597 598

    //Add log files in wal_dir
599
    if (db_options_.wal_dir != dbname_) {
600
      std::vector<std::string> log_files;
601
      env_->GetChildren(db_options_.wal_dir, &log_files);  // Ignore errors
602
      for (std::string log_file : log_files) {
I
Igor Canadi 已提交
603
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
604 605
      }
    }
606
    // Add info log files in db_log_dir
607
    if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) {
608
      std::vector<std::string> info_log_files;
609 610
      // Ignore errors
      env_->GetChildren(db_options_.db_log_dir, &info_log_files);
611
      for (std::string log_file : info_log_files) {
I
Igor Canadi 已提交
612
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
613 614
      }
    }
615
  }
616 617
}

618
namespace {
I
Igor Canadi 已提交
619 620
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
621 622 623 624 625
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
626
    return (first.path_id > second.path_id);
627 628 629 630
  }
}
};  // namespace

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

I
Igor Canadi 已提交
639 640 641 642
  // 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 已提交
643 644
    return;
  }
645

646
  // Now, convert live list to an unordered map, WITHOUT mutex held;
647
  // set is slow.
648
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
I
Igor Canadi 已提交
649
  for (const FileDescriptor& fd : state.sst_live) {
650 651
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
652

I
Igor Canadi 已提交
653
  auto candidate_files = state.full_scan_candidate_files;
I
Igor Canadi 已提交
654 655 656
  candidate_files.reserve(candidate_files.size() +
                          state.sst_delete_files.size() +
                          state.log_delete_files.size());
K
kailiu 已提交
657 658
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
659
  for (auto file : state.sst_delete_files) {
660 661 662
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
663
    delete file;
I
Igor Canadi 已提交
664 665
  }

K
kailiu 已提交
666 667
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
668 669
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
670 671
    }
  }
672

K
kailiu 已提交
673
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
674
  // file twice
675
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
676 677
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
678

679
  std::vector<std::string> old_info_log_files;
680
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
681 682 683
  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 已提交
684 685 686
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
687
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
688 689
      continue;
    }
J
jorlow@chromium.org 已提交
690

K
kailiu 已提交
691 692 693 694 695 696 697 698
    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'
699
        // (can happen during manifest roll)
K
kailiu 已提交
700 701 702
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
I
Igor Canadi 已提交
703 704 705 706
        // 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 已提交
707 708 709
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
710 711 712 713
        // 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
714
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
715
               (number == state.pending_manifest_file_number);
K
kailiu 已提交
716 717 718 719
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
720
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
721
        }
K
kailiu 已提交
722 723 724 725 726 727 728 729 730 731 732 733 734
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

735
    std::string fname;
K
kailiu 已提交
736 737
    if (type == kTableFile) {
      // evict from cache
738
      TableCache::Evict(table_cache_.get(), number);
739
      fname = TableFileName(db_options_.db_paths, number, path_id);
740
    } else {
741 742
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
743
    }
744

745
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
746
    if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 ||
747
                              db_options_.WAL_size_limit_MB > 0)) {
I
Igor Canadi 已提交
748
      wal_manager_.ArchiveWALFile(fname, number);
749 750 751
      continue;
    }
#endif  // !ROCKSDB_LITE
I
Islam AbdelRahman 已提交
752 753 754 755 756 757 758
    Status file_deletion_status;
    if (db_options_.delete_scheduler != nullptr && type == kTableFile &&
        path_id == 0) {
      file_deletion_status = db_options_.delete_scheduler->DeleteFile(fname);
    } else {
      file_deletion_status = env_->DeleteFile(fname);
    }
759
    if (file_deletion_status.ok()) {
760
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
761
          "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", state.job_id,
762 763 764 765 766 767 768 769 770
          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) {
771 772 773 774
      EventHelpers::LogAndNotifyTableFileDeletion(
          &event_logger_, state.job_id, number, fname,
          file_deletion_status, GetName(),
          db_options_.listeners);
J
jorlow@chromium.org 已提交
775 776
    }
  }
H
heyongqiang 已提交
777

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

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
806
  JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
807
  FindObsoleteFiles(&job_context, true);
808 809

  mutex_.Unlock();
I
Igor Canadi 已提交
810 811
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
812
  }
I
Igor Canadi 已提交
813
  job_context.Clean();
814
  mutex_.Lock();
815 816
}

817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832
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 已提交
833

834 835 836 837 838 839 840 841 842
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_);
843 844 845
    if (!s.ok()) {
      return s;
    }
846
  }
847

848 849 850 851 852 853 854 855
  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);
856 857 858
      if (!s.ok()) {
        return s;
      }
859
      data_dirs_.emplace_back(path_directory.release());
860
    }
861 862 863 864 865 866 867 868 869 870 871 872 873 874 875 876 877 878 879
  }
  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();
880

881 882 883 884 885
  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);
886 887 888 889
    if (!s.ok()) {
      return s;
    }

890
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
891 892 893
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
894

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

930
  Status s = versions_->Recover(column_families, read_only);
931
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
932 933
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
934
  if (s.ok()) {
A
agiardullo 已提交
935
    SequenceNumber max_sequence(kMaxSequenceNumber);
936 937
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
938
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
939 940
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
941 942 943 944 945

    // 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).
    //
946
    // Note that prev_log_number() is no longer used, but we pay
947
    // attention to it in case we are recovering a database
948
    // produced by an older version of rocksdb.
949
    const uint64_t min_log = versions_->MinLogNumber();
950
    const uint64_t prev_log = versions_->prev_log_number();
951
    std::vector<std::string> filenames;
952
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
953 954
    if (!s.ok()) {
      return s;
955
    }
K
kailiu 已提交
956

957 958
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
959 960
      uint64_t number;
      FileType type;
961 962 963 964 965 966 967 968 969
      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);
        }
970
      }
J
jorlow@chromium.org 已提交
971
    }
972

H
heyongqiang 已提交
973 974 975 976 977 978
    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 已提交
979 980 981 982 983 984 985
    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 已提交
986 987
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 kMaxSequenceNumber);
S
Stanislau Hlebik 已提交
988 989
        }
      }
990
    }
L
Lei Jin 已提交
991
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
992 993
  }

L
Lei Jin 已提交
994 995
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
996
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
997 998 999
    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 已提交
1000 1001
  }

J
jorlow@chromium.org 已提交
1002 1003 1004
  return s;
}

S
Stanislau Hlebik 已提交
1005 1006 1007
// 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 已提交
1008 1009
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
1010
    Logger* info_log;
J
jorlow@chromium.org 已提交
1011
    const char* fname;
1012
    Status* status;  // nullptr if db_options_.paranoid_checks==false
I
Igor Sugak 已提交
1013
    virtual void Corruption(size_t bytes, const Status& s) override {
1014 1015
      Log(InfoLogLevel::WARN_LEVEL,
          info_log, "%s%s: dropping %d bytes; %s",
1016
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
1017
          fname, static_cast<int>(bytes), s.ToString().c_str());
K
krad 已提交
1018 1019 1020
      if (this->status != nullptr && this->status->ok()) {
        *this->status = s;
      }
J
jorlow@chromium.org 已提交
1021 1022 1023 1024
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
1025
  Status status;
1026
  std::unordered_map<int, VersionEdit> version_edits;
1027
  // no need to refcount because iteration is under mutex
1028 1029
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1030 1031
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1032
  }
1033 1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044
  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 已提交
1045

K
krad 已提交
1046
  bool continue_replay_log = true;
S
Stanislau Hlebik 已提交
1047 1048 1049 1050
  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.
1051
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
1052 1053
    // Open the log file
    std::string fname = LogFileName(db_options_.wal_dir, log_number);
1054 1055 1056 1057
    unique_ptr<SequentialFileReader> file_reader;
    {
      unique_ptr<SequentialFile> file;
      status = env_->NewSequentialFile(fname, &file, env_options_);
S
Stanislau Hlebik 已提交
1058
      if (!status.ok()) {
1059 1060 1061 1062 1063 1064 1065 1066
        MaybeIgnoreError(&status);
        if (!status.ok()) {
          return status;
        } else {
          // Fail with one log file, but that's ok.
          // Try next one.
          continue;
        }
S
Stanislau Hlebik 已提交
1067
      }
1068
      file_reader.reset(new SequentialFileReader(std::move(file)));
J
jorlow@chromium.org 已提交
1069 1070
    }

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

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

      // TODO krad: Evaluate if we need to move to a more strict mode where we
      // restrict the inconsistency to only the last log
    }
S
Stanislau Hlebik 已提交
1108 1109 1110 1111 1112

    // Read all the records and add to a memtable
    std::string scratch;
    Slice record;
    WriteBatch batch;
K
krad 已提交
1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125

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

    while (continue_replay_log &&
           reader.ReadRecord(&record, &scratch, report_eof_inconsistency) &&
           status.ok()) {
S
Stanislau Hlebik 已提交
1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146
      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 已提交
1147
      if ((*max_sequence == kMaxSequenceNumber) || (last_seq > *max_sequence)) {
S
Stanislau Hlebik 已提交
1148 1149 1150 1151
        *max_sequence = last_seq;
      }

      if (!read_only) {
I
Igor Canadi 已提交
1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163
        // 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;
1164
          status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
I
Igor Canadi 已提交
1165 1166 1167 1168
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
1169
          }
A
agiardullo 已提交
1170 1171 1172

          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 *max_sequence);
1173
        }
J
jorlow@chromium.org 已提交
1174 1175 1176
      }
    }

1177
    if (!status.ok()) {
K
krad 已提交
1178 1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195
      // The hook function is designed to ignore all IO errors from reader
      // during recovery for kSkipAnyCorruptedRecords. Status variable is
      // unmodified by the reader.
      assert(db_options_.wal_recovery_mode !=
             WALRecoveryMode::kSkipAnyCorruptedRecords);
      if (db_options_.wal_recovery_mode ==
                 WALRecoveryMode::kPointInTimeRecovery) {
        // We should ignore the error but not continue replaying
        status = Status::OK();
        continue_replay_log = false;

        Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
            "Point in time recovered to log #%" PRIu64 " seq #%" PRIu64,
            log_number, *max_sequence);
      } else if (db_options_.wal_recovery_mode !=
                 WALRecoveryMode::kSkipAnyCorruptedRecords) {
        return status;
      }
1196 1197
    }

I
Igor Canadi 已提交
1198
    flush_scheduler_.Clear();
A
agiardullo 已提交
1199 1200
    if ((*max_sequence != kMaxSequenceNumber) &&
        (versions_->LastSequence() < *max_sequence)) {
S
Stanislau Hlebik 已提交
1201 1202
      versions_->SetLastSequence(*max_sequence);
    }
1203 1204
  }

1205
  if (!read_only) {
1206 1207
    // 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 已提交
1208
    auto max_log_number = log_numbers.back();
1209
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1210
      auto iter = version_edits.find(cfd->GetID());
1211 1212 1213
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
1214
      if (cfd->GetLogNumber() > max_log_number) {
1215
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
1216
        // from all logs. Memtable has to be empty because
1217
        // we filter the updates based on log_number
1218
        // (in WriteBatch::InsertInto)
1219 1220 1221 1222 1223 1224 1225
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1226
        status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
1227 1228 1229 1230
        if (!status.ok()) {
          // Recovery failed
          break;
        }
A
agiardullo 已提交
1231 1232 1233

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

1236
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
1237
      // writing log_number in the manifest means that any log file
1238 1239
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
1240 1241 1242
      // 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);
1243 1244 1245
      // 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 已提交
1246
      // log number
1247
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
1248 1249
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
1250
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1251 1252
        // Recovery failed
        break;
1253 1254
      }
    }
1255
  }
I
Igor Canadi 已提交
1256

1257 1258 1259
  event_logger_.Log() << "job" << job_id << "event"
                      << "recovery_finished";

J
jorlow@chromium.org 已提交
1260 1261 1262
  return status;
}

1263 1264
Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
                                           MemTable* mem, VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1265
  mutex_.AssertHeld();
1266
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1267
  FileMetaData meta;
1268
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1269 1270
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1271 1272
  ReadOptions ro;
  ro.total_order_seek = true;
1273
  Arena arena;
1274
  Status s;
1275
  TableProperties table_properties;
1276
  {
1277 1278 1279 1280
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
    const SequenceNumber newest_snapshot = snapshots_.GetNewest();
    const SequenceNumber earliest_seqno_in_memtable =
        mem->GetFirstSequenceNumber();
1281 1282 1283
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1284
        cfd->GetName().c_str(), meta.fd.GetNumber());
1285

1286 1287
    bool paranoid_file_checks =
        cfd->GetLatestMutableCFOptions()->paranoid_file_checks;
1288 1289
    {
      mutex_.Unlock();
1290
      TableFileCreationInfo info;
1291 1292
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
1293 1294
          iter.get(), &meta, cfd->internal_comparator(),
          cfd->int_tbl_prop_collector_factories(), newest_snapshot,
1295
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
1296 1297
          cfd->ioptions()->compression_opts, paranoid_file_checks,
          cfd->internal_stats(), Env::IO_HIGH, &info.table_properties);
1298
      LogFlush(db_options_.info_log);
1299 1300 1301 1302 1303 1304 1305 1306
      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()) {
1307 1308 1309 1310 1311 1312 1313 1314 1315
        info.db_name = dbname_;
        info.cf_name = cfd->GetName();
        info.file_path = TableFileName(db_options_.db_paths,
                                       meta.fd.GetNumber(),
                                       meta.fd.GetPathId());
        info.file_size = meta.fd.GetFileSize();
        info.job_id = job_id;
        EventHelpers::LogAndNotifyTableFileCreation(
            &event_logger_, db_options_.listeners, meta.fd, info);
1316
      }
1317 1318
      mutex_.Lock();
    }
1319
  }
I
Igor Canadi 已提交
1320
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1321 1322 1323 1324

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1325
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1326 1327
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
1328 1329
                  meta.smallest_seqno, meta.largest_seqno,
                  meta.marked_for_compaction);
1330 1331
  }

L
Lei Jin 已提交
1332
  InternalStats::CompactionStats stats(1);
1333
  stats.micros = env_->NowMicros() - start_micros;
1334
  stats.bytes_written = meta.fd.GetFileSize();
1335
  stats.num_output_files = 1;
1336
  cfd->internal_stats()->AddCompactionStats(level, stats);
1337 1338
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1339
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1340 1341 1342
  return s;
}

1343 1344
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
I
Igor Canadi 已提交
1345
    bool* madeProgress, JobContext* job_context, LogBuffer* log_buffer) {
1346
  mutex_.AssertHeld();
1347
  assert(cfd->imm()->NumNotFlushed() != 0);
1348
  assert(cfd->imm()->IsFlushPending());
1349

I
Igor Canadi 已提交
1350 1351
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1352
                     snapshots_.GetNewest(), job_context, log_buffer,
1353
                     directories_.GetDbDir(), directories_.GetDataDir(0U),
I
Igor Canadi 已提交
1354 1355
                     GetCompressionFlush(*cfd->ioptions()), stats_,
                     &event_logger_);
1356

1357
  FileMetaData file_meta;
1358 1359 1360 1361 1362 1363 1364

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

  if (s.ok()) {
I
Igor Canadi 已提交
1368 1369
    InstallSuperVersionAndScheduleWorkWrapper(cfd, job_context,
                                              mutable_cf_options);
1370 1371 1372
    if (madeProgress) {
      *madeProgress = 1;
    }
S
sdong 已提交
1373
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1374
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1375
                cfd->current()->storage_info()->LevelSummary(&tmp));
J
jorlow@chromium.org 已提交
1376
  }
1377

1378
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1379 1380 1381 1382 1383
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1384
  RecordFlushIOStats();
1385 1386 1387
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1388
    NotifyOnFlushCompleted(cfd, &file_meta, mutable_cf_options,
1389
                           job_context->job_id);
1390 1391
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1392 1393 1394
  return s;
}

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

1437 1438 1439 1440
Status DBImpl::CompactRange(const CompactRangeOptions& options,
                            ColumnFamilyHandle* column_family,
                            const Slice* begin, const Slice* end) {
  if (options.target_path_id >= db_options_.db_paths.size()) {
1441 1442 1443
    return Status::InvalidArgument("Invalid target path ID");
  }

1444 1445
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1446 1447

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1448
  if (!s.ok()) {
1449
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1450 1451 1452
    return s;
  }

I
Igor Canadi 已提交
1453
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1454
  {
1455
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
1456
    Version* base = cfd->current();
1457 1458
    for (int level = 1; level < base->storage_info()->num_non_empty_levels();
         level++) {
S
sdong 已提交
1459
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1460 1461 1462 1463
        max_level_with_files = level;
      }
    }
  }
1464

1465
  int final_output_level = 0;
1466 1467 1468 1469
  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
1470 1471
                            cfd->NumberLevels() - 1, options.target_path_id,
                            begin, end);
1472
    final_output_level = cfd->NumberLevels() - 1;
1473 1474
  } else {
    for (int level = 0; level <= max_level_with_files; level++) {
1475
      int output_level;
1476 1477 1478 1479 1480
      // 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 ||
1481 1482 1483
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
        output_level = level;
      } else if (level == max_level_with_files && level > 0) {
1484 1485 1486 1487 1488 1489 1490
        if (options.bottommost_level_compaction ==
            BottommostLevelCompaction::kSkip) {
          // Skip bottommost level compaction
          continue;
        } else if (options.bottommost_level_compaction ==
                       BottommostLevelCompaction::kIfHaveCompactionFilter &&
                   cfd->ioptions()->compaction_filter == nullptr &&
I
Igor Canadi 已提交
1491
                   cfd->ioptions()->compaction_filter_factory == nullptr) {
1492 1493
          // Skip bottommost level compaction since we dont have
          // compaction filter
1494 1495
          continue;
        }
1496
        output_level = level;
1497
      } else {
1498
        output_level = level + 1;
1499 1500 1501 1502 1503
        if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
            cfd->ioptions()->level_compaction_dynamic_level_bytes &&
            level == 0) {
          output_level = ColumnFamilyData::kCompactToBaseLevel;
        }
1504
      }
1505 1506
      s = RunManualCompaction(cfd, level, output_level, options.target_path_id,
                              begin, end);
1507 1508 1509
      if (!s.ok()) {
        break;
      }
1510 1511 1512 1513 1514
      if (output_level == ColumnFamilyData::kCompactToBaseLevel) {
        final_output_level = cfd->NumberLevels() - 1;
      } else if (output_level > final_output_level) {
        final_output_level = output_level;
      }
1515 1516
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
1517
    }
G
Gabor Cselle 已提交
1518
  }
1519 1520 1521 1522
  if (!s.ok()) {
    LogFlush(db_options_.info_log);
    return s;
  }
1523

1524 1525
  if (options.change_level) {
    s = ReFitLevel(cfd, final_output_level, options.target_level);
1526
  }
1527
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1528

1529
  {
1530
    InstrumentedMutexLock l(&mutex_);
1531 1532 1533 1534 1535
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1536
  return s;
1537 1538
}

1539 1540 1541 1542 1543
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 已提交
1544 1545 1546 1547
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1548 1549 1550 1551 1552 1553
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);
1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576 1577 1578

  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());
1579
  }  // release the mutex
1580 1581 1582 1583 1584 1585 1586 1587 1588 1589

  // 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()) {
1590
      // no mutex is locked here.  No need to Unlock() and Lock() here.
1591 1592 1593 1594 1595
      PurgeObsoleteFiles(job_context);
    }
    job_context.Clean();
  }

1596
  return s;
I
Igor Canadi 已提交
1597
#endif  // ROCKSDB_LITE
1598 1599
}

I
Igor Canadi 已提交
1600
#ifndef ROCKSDB_LITE
1601 1602 1603
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
1604 1605
    const int output_level, int output_path_id, JobContext* job_context,
    LogBuffer* log_buffer) {
1606 1607 1608 1609 1610 1611 1612 1613 1614 1615 1616 1617 1618 1619 1620 1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637
  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;
  }

1638
  std::vector<CompactionInputFiles> input_files;
1639 1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651 1652 1653 1654 1655 1656 1657 1658
  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(
1659 1660
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
1661 1662 1663
  assert(c);
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
1664
  assert(!c->deletion_compaction());
1665

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

  mutex_.Unlock();
1690
  compaction_job.Run();
1691
  mutex_.Lock();
1692

1693
  Status status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
1694
  if (status.ok()) {
I
Igor Canadi 已提交
1695 1696
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
1697 1698 1699 1700 1701 1702 1703 1704 1705
  }
  c->ReleaseCompactionFiles(s);
  c.reset();

  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
1706 1707
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] [JOB %d] Compaction error: %s",
1708
        c->column_family_data()->GetName().c_str(), job_context->job_id,
1709 1710 1711 1712 1713 1714 1715
        status.ToString().c_str());
    if (db_options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }

  bg_compaction_scheduled_--;
1716 1717 1718
  if (bg_compaction_scheduled_ == 0) {
    bg_cv_.SignalAll();
  }
1719 1720 1721

  return status;
}
I
Igor Canadi 已提交
1722
#endif  // ROCKSDB_LITE
1723

O
Ori Bernstein 已提交
1724
void DBImpl::NotifyOnCompactionCompleted(
1725 1726
    ColumnFamilyData* cfd, Compaction *c, const Status &st,
    const CompactionJobStats& compaction_job_stats,
1727
    const int job_id) {
O
Ori Bernstein 已提交
1728
#ifndef ROCKSDB_LITE
1729
  if (db_options_.listeners.size() == 0U) {
O
Ori Bernstein 已提交
1730 1731 1732 1733 1734 1735 1736 1737
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  // release lock while notifying events
  mutex_.Unlock();
1738 1739 1740 1741
  {
    CompactionJobInfo info;
    info.cf_name = cfd->GetName();
    info.status = st;
1742
    info.thread_id = env_->GetThreadID();
1743 1744
    info.job_id = job_id;
    info.base_input_level = c->start_level();
1745
    info.output_level = c->output_level();
1746
    info.stats = compaction_job_stats;
1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764
    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 已提交
1765 1766 1767 1768 1769 1770
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
#endif  // ROCKSDB_LITE
}

1771
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1772
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1773 1774 1775
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1776 1777
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1778 1779
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1780
        cfd->GetName().c_str());
1781
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1782 1783 1784
  }

  MutableCFOptions new_options;
1785
  Status s;
L
Lei Jin 已提交
1786
  {
1787
    InstrumentedMutexLock l(&mutex_);
1788 1789
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1790 1791 1792 1793
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

1794 1795
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1796 1797
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1798 1799
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1800
  }
1801
  if (s.ok()) {
1802 1803
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1804 1805 1806
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
  } else {
1807 1808
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1809
  }
S
sdong 已提交
1810
  LogFlush(db_options_.info_log);
1811
  return s;
I
Igor Canadi 已提交
1812
#endif  // ROCKSDB_LITE
1813 1814
}

1815
// return the same level if it cannot be moved
1816 1817
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
1818
  mutex_.AssertHeld();
S
sdong 已提交
1819
  const auto* vstorage = cfd->current()->storage_info();
1820
  int minimum_level = level;
1821
  for (int i = level - 1; i > 0; --i) {
1822
    // stop if level i is not empty
S
sdong 已提交
1823
    if (vstorage->NumLevelFiles(i) > 0) break;
1824
    // stop if level i is too small (cannot fit the level files)
1825
    if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) {
1826 1827
      break;
    }
1828 1829 1830 1831 1832 1833

    minimum_level = i;
  }
  return minimum_level;
}

I
Igor Canadi 已提交
1834 1835
Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
  assert(level < cfd->NumberLevels());
1836 1837 1838
  if (target_level >= cfd->NumberLevels()) {
    return Status::InvalidArgument("Target level exceeds number of levels");
  }
1839

I
Igor Canadi 已提交
1840
  SuperVersion* superversion_to_free = nullptr;
1841
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1842

I
Islam AbdelRahman 已提交
1843
  InstrumentedMutexLock guard_lock(&mutex_);
1844 1845 1846

  // only allow one thread refitting
  if (refitting_level_) {
1847 1848
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
I
Igor Canadi 已提交
1849
    delete new_superversion;
L
Lei Jin 已提交
1850
    return Status::NotSupported("another thread is refitting");
1851 1852 1853 1854 1855
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1856
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1857 1858
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop: %d %d",
1859
        bg_compaction_scheduled_, bg_flush_scheduled_);
1860 1861 1862
    bg_cv_.Wait();
  }

1863 1864
  const MutableCFOptions mutable_cf_options =
    *cfd->GetLatestMutableCFOptions();
1865
  // move to a smaller level
1866 1867
  int to_level = target_level;
  if (target_level < 0) {
1868
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1869
  }
1870

L
Lei Jin 已提交
1871
  Status status;
1872 1873 1874
  auto* vstorage = cfd->current()->storage_info();
  if (to_level > level) {
    if (level == 0) {
I
Islam AbdelRahman 已提交
1875
      delete new_superversion;
1876 1877 1878 1879 1880 1881
      return Status::NotSupported(
          "Cannot change from level 0 to other levels.");
    }
    // Check levels are empty for a trivial move
    for (int l = level + 1; l <= to_level; l++) {
      if (vstorage->NumLevelFiles(l) > 0) {
I
Islam AbdelRahman 已提交
1882
        delete new_superversion;
1883 1884 1885 1886 1887 1888
        return Status::NotSupported(
            "Levels between source and target are not empty for a move.");
      }
    }
  }
  if (to_level != level) {
1889 1890
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Before refitting:\n%s",
1891
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1892

1893
    VersionEdit edit;
I
Igor Canadi 已提交
1894
    edit.SetColumnFamily(cfd->GetID());
1895
    for (const auto& f : vstorage->LevelFiles(level)) {
1896
      edit.DeleteFile(level, f->fd.GetNumber());
1897 1898
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
1899 1900
                   f->smallest_seqno, f->largest_seqno,
                   f->marked_for_compaction);
1901
    }
1902 1903
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1904
        cfd->GetName().c_str(), edit.DebugString().data());
1905

1906 1907
    status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                    directories_.GetDbDir());
I
Igor Canadi 已提交
1908
    superversion_to_free = InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
1909
        cfd, new_superversion, mutable_cf_options);
I
Igor Canadi 已提交
1910
    new_superversion = nullptr;
1911

1912 1913
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1914
        status.ToString().data());
1915 1916

    if (status.ok()) {
1917 1918
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1919
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1920 1921 1922 1923 1924
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1925 1926 1927

  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1928
  return status;
G
Gabor Cselle 已提交
1929 1930
}

1931 1932 1933
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1934 1935
}

1936
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
1937
  return 0;
1938 1939
}

1940 1941
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1942
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
1943 1944
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
1945 1946
}

L
Lei Jin 已提交
1947
Status DBImpl::Flush(const FlushOptions& flush_options,
1948 1949
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1950
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
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
Status DBImpl::SyncWAL() {
  autovector<log::Writer*, 1> logs_to_sync;
  bool need_log_dir_sync;
  uint64_t current_log_number;

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

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

    while (logs_.front().number <= current_log_number &&
           logs_.front().getting_synced) {
      log_sync_cv_.Wait();
    }
    // First check that logs are safe to sync in background.
    for (auto it = logs_.begin();
         it != logs_.end() && it->number <= current_log_number; ++it) {
      if (!it->writer->file()->writable_file()->IsSyncThreadSafe()) {
        return Status::NotSupported(
          "SyncWAL() is not supported for this implementation of WAL file",
          db_options_.allow_mmap_writes
            ? "try setting Options::allow_mmap_writes to false"
            : Slice());
      }
    }
    for (auto it = logs_.begin();
         it != logs_.end() && it->number <= current_log_number; ++it) {
      auto& log = *it;
      assert(!log.getting_synced);
      log.getting_synced = true;
1985
      logs_to_sync.push_back(log.writer);
1986 1987 1988 1989 1990 1991 1992 1993 1994 1995 1996 1997 1998 1999 2000 2001 2002 2003 2004 2005 2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017 2018 2019 2020 2021
    }

    need_log_dir_sync = !log_dir_synced_;
  }

  Status status;
  for (log::Writer* log : logs_to_sync) {
    status = log->file()->SyncWithoutFlush(db_options_.use_fsync);
    if (!status.ok()) {
      break;
    }
  }
  if (status.ok() && need_log_dir_sync) {
    status = directories_.GetWalDir()->Fsync();
  }

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

  return status;
}

void DBImpl::MarkLogsSynced(
    uint64_t up_to, bool synced_dir, const Status& status) {
  mutex_.AssertHeld();
  if (synced_dir &&
      logfile_number_ == up_to &&
      status.ok()) {
    log_dir_synced_ = true;
  }
  for (auto it = logs_.begin(); it != logs_.end() && it->number <= up_to;) {
    auto& log = *it;
    assert(log.getting_synced);
    if (status.ok() && logs_.size() > 1) {
2022
      logs_to_free_.push_back(log.ReleaseWriter());
2023 2024 2025 2026 2027 2028 2029 2030 2031
      logs_.erase(it++);
    } else {
      log.getting_synced = false;
      ++it;
    }
  }
  log_sync_cv_.SignalAll();
}

2032
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
2033 2034 2035
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
2036
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
2037
                                   int output_level, uint32_t output_path_id,
2038 2039
                                   const Slice* begin, const Slice* end,
                                   bool disallow_trivial_move) {
2040 2041
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);
2042

G
Gabor Cselle 已提交
2043 2044
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
2045
  ManualCompaction manual;
I
Igor Canadi 已提交
2046
  manual.cfd = cfd;
2047 2048
  manual.input_level = input_level;
  manual.output_level = output_level;
2049
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
2050
  manual.done = false;
2051
  manual.in_progress = false;
2052
  manual.disallow_trivial_move = disallow_trivial_move;
2053 2054 2055
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
2056 2057
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2058
    manual.begin = nullptr;
G
Gabor Cselle 已提交
2059
  } else {
2060
    begin_storage.SetMaxPossibleForUserKey(*begin);
G
Gabor Cselle 已提交
2061 2062
    manual.begin = &begin_storage;
  }
2063
  if (end == nullptr ||
2064 2065
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2066
    manual.end = nullptr;
G
Gabor Cselle 已提交
2067
  } else {
2068
    end_storage.SetMinPossibleForUserKey(*end);
G
Gabor Cselle 已提交
2069 2070 2071
    manual.end = &end_storage;
  }

2072
  InstrumentedMutexLock l(&mutex_);
2073

2074 2075 2076 2077 2078 2079
  // 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
2080
  // RunManualCompaction(), i.e. during that time no other compaction will
2081 2082 2083
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
2084
  // RunManualCompaction() from getting to the second while loop below.
2085 2086 2087 2088 2089
  // 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) {
2090
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
2091 2092 2093
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
2094 2095
    bg_cv_.Wait();
  }
2096

2097 2098
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
2099
      cfd->GetName().c_str());
2100

2101 2102 2103 2104
  // 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) {
2105 2106 2107
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
2108
      bg_cv_.Wait();
2109 2110
    } else {
      manual_compaction_ = &manual;
2111
      bg_compaction_scheduled_++;
2112
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
G
Gabor Cselle 已提交
2113
    }
H
hans@chromium.org 已提交
2114
  }
2115

2116 2117 2118
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
2119
  return manual.status;
J
jorlow@chromium.org 已提交
2120 2121
}

2122
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
2123
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
2124 2125 2126
  Status s;
  {
    WriteContext context;
2127
    InstrumentedMutexLock guard_lock(&mutex_);
2128

2129
    if (cfd->imm()->NumNotFlushed() == 0 && cfd->mem()->IsEmpty()) {
2130 2131 2132 2133
      // Nothing to flush
      return Status::OK();
    }

2134 2135
    WriteThread::Writer w;
    write_thread_.EnterUnbatched(&w, &mutex_);
S
Stanislau Hlebik 已提交
2136

I
Igor Canadi 已提交
2137
    // SwitchMemtable() will release and reacquire mutex
S
Stanislau Hlebik 已提交
2138
    // during execution
I
Igor Canadi 已提交
2139
    s = SwitchMemtable(cfd, &context);
2140
    write_thread_.ExitUnbatched(&w);
2141

S
Stanislau Hlebik 已提交
2142 2143
    cfd->imm()->FlushRequested();

2144 2145 2146
    // schedule flush
    SchedulePendingFlush(cfd);
    MaybeScheduleFlushOrCompaction();
S
Stanislau Hlebik 已提交
2147
  }
S
Stanislau Hlebik 已提交
2148

L
Lei Jin 已提交
2149
  if (s.ok() && flush_options.wait) {
2150
    // Wait until the compaction completes
2151
    s = WaitForFlushMemTable(cfd);
2152 2153
  }
  return s;
J
jorlow@chromium.org 已提交
2154 2155
}

2156
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
2157 2158
  Status s;
  // Wait until the compaction completes
2159
  InstrumentedMutexLock l(&mutex_);
2160
  while (cfd->imm()->NumNotFlushed() > 0 && bg_error_.ok()) {
2161 2162 2163
    if (shutting_down_.load(std::memory_order_acquire)) {
      return Status::ShutdownInProgress();
    }
2164 2165
    bg_cv_.Wait();
  }
2166
  if (!bg_error_.ok()) {
2167 2168 2169
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
2170 2171
}

2172
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
2173
  mutex_.AssertHeld();
2174 2175 2176 2177
  if (!opened_successfully_) {
    // Compaction may introduce data race to DB open
    return;
  }
2178
  if (bg_work_gate_closed_) {
2179 2180
    // gate closed for background work
    return;
I
Igor Canadi 已提交
2181
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
2182
    // DB is being deleted; no more background compactions
2183 2184
    return;
  }
2185

2186 2187 2188 2189
  while (unscheduled_flushes_ > 0 &&
         bg_flush_scheduled_ < db_options_.max_background_flushes) {
    unscheduled_flushes_--;
    bg_flush_scheduled_++;
2190
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
2191 2192
  }

2193 2194 2195 2196 2197 2198 2199 2200 2201 2202 2203 2204
  // special case -- if max_background_flushes == 0, then schedule flush on a
  // compaction thread
  if (db_options_.max_background_flushes == 0) {
    while (unscheduled_flushes_ > 0 &&
           bg_flush_scheduled_ + bg_compaction_scheduled_ <
               db_options_.max_background_compactions) {
      unscheduled_flushes_--;
      bg_flush_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::LOW, this);
    }
  }

2205 2206 2207 2208 2209 2210
  if (bg_manual_only_) {
    // only manual compactions are allowed to run. don't schedule automatic
    // compactions
    return;
  }

2211 2212 2213 2214
  while (bg_compaction_scheduled_ < db_options_.max_background_compactions &&
         unscheduled_compactions_ > 0) {
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
2215
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
2216 2217 2218 2219 2220 2221 2222 2223 2224 2225 2226 2227 2228 2229 2230 2231 2232 2233 2234 2235 2236 2237 2238 2239 2240 2241 2242 2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255 2256 2257 2258 2259 2260 2261
  }
}

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_;
2262 2263 2264
  }
}

2265
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
2266
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
2267 2268 2269
  IOSTATS_RESET(bytes_written);
}

2270
void DBImpl::BGWorkFlush(void* db) {
2271
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
2272
  TEST_SYNC_POINT("DBImpl::BGWorkFlush");
2273
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
2274
  TEST_SYNC_POINT("DBImpl::BGWorkFlush:done");
2275 2276 2277
}

void DBImpl::BGWorkCompaction(void* db) {
2278
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
2279
  TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
2280 2281 2282
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
2283
Status DBImpl::BackgroundFlush(bool* madeProgress, JobContext* job_context,
H
Haobo Xu 已提交
2284
                               LogBuffer* log_buffer) {
2285
  mutex_.AssertHeld();
2286

2287 2288 2289 2290 2291 2292 2293
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
    return status;
2294 2295
  }

2296 2297 2298
  ColumnFamilyData* cfd = nullptr;
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
I
Igor Canadi 已提交
2299
    auto first_cfd = PopFirstFromFlushQueue();
2300

I
Igor Canadi 已提交
2301
    if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
2302
      // can't flush this CF, try next one
I
Igor Canadi 已提交
2303 2304
      if (first_cfd->Unref()) {
        delete first_cfd;
2305 2306
      }
      continue;
2307
    }
2308 2309

    // found a flush!
I
Igor Canadi 已提交
2310
    cfd = first_cfd;
2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326 2327
    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;
2328
    }
J
jorlow@chromium.org 已提交
2329
  }
2330
  return status;
J
jorlow@chromium.org 已提交
2331 2332
}

2333
void DBImpl::BackgroundCallFlush() {
2334
  bool madeProgress = false;
2335
  JobContext job_context(next_job_id_.fetch_add(1), true);
2336 2337
  assert(bg_flush_scheduled_);

2338
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2339
  {
2340
    InstrumentedMutexLock l(&mutex_);
H
Haobo Xu 已提交
2341

I
Igor Canadi 已提交
2342 2343 2344
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2345
    Status s = BackgroundFlush(&madeProgress, &job_context, &log_buffer);
2346 2347 2348 2349 2350 2351 2352 2353 2354 2355 2356 2357 2358 2359 2360 2361 2362
    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 已提交
2363 2364
    }

I
Igor Canadi 已提交
2365 2366
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2367 2368 2369
    // 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 已提交
2370
    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2371
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2372
      mutex_.Unlock();
2373 2374 2375 2376 2377
      // 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 已提交
2378
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2379 2380
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2381
      }
I
Igor Canadi 已提交
2382
      job_context.Clean();
2383 2384
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2385

H
Haobo Xu 已提交
2386
    bg_flush_scheduled_--;
2387 2388
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2389
    RecordFlushIOStats();
H
Haobo Xu 已提交
2390
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2391 2392 2393 2394
    // 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.
2395
  }
J
jorlow@chromium.org 已提交
2396 2397
}

2398
void DBImpl::BackgroundCallCompaction() {
2399
  bool madeProgress = false;
2400
  JobContext job_context(next_job_id_.fetch_add(1), true);
H
Haobo Xu 已提交
2401 2402

  MaybeDumpStats();
2403
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2404
  {
2405
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
2406 2407 2408 2409

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2410
    assert(bg_compaction_scheduled_);
2411
    Status s = BackgroundCompaction(&madeProgress, &job_context, &log_buffer);
2412 2413 2414 2415 2416 2417
    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 =
2418
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2419 2420 2421 2422 2423 2424 2425 2426 2427 2428
      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();
2429
    }
H
Haobo Xu 已提交
2430

I
Igor Canadi 已提交
2431 2432
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2433 2434 2435 2436
    // 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());
2437 2438

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2439
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2440
      mutex_.Unlock();
2441 2442 2443 2444 2445
      // 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 已提交
2446
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2447 2448
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2449
      }
I
Igor Canadi 已提交
2450
      job_context.Clean();
2451 2452
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2453

2454
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2455

2456 2457
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2458 2459
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2460 2461
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
I
Igor Canadi 已提交
2462
      // * madeProgress -- need to wakeup DelayWrite
2463 2464 2465 2466 2467 2468
      // * 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 已提交
2469 2470 2471 2472
    // 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.
2473
  }
J
jorlow@chromium.org 已提交
2474 2475
}

I
Igor Canadi 已提交
2476
Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
2477
                                    LogBuffer* log_buffer) {
2478
  *madeProgress = false;
J
jorlow@chromium.org 已提交
2479
  mutex_.AssertHeld();
2480

2481 2482
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2483 2484
  bool trivial_move_disallowed = is_manual &&
                                 manual_compaction_->disallow_trivial_move;
2485

2486
  CompactionJobStats compaction_job_stats;
2487 2488 2489 2490 2491 2492
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
2493
    if (is_manual) {
2494
      manual_compaction_->status = status;
2495 2496 2497 2498
      manual_compaction_->done = true;
      manual_compaction_->in_progress = false;
      manual_compaction_ = nullptr;
    }
2499
    return status;
2500 2501
  }

2502 2503 2504
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2505 2506 2507 2508
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2509 2510 2511
  }

  unique_ptr<Compaction> c;
2512 2513
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2514
  if (is_manual) {
G
Gabor Cselle 已提交
2515
    ManualCompaction* m = manual_compaction_;
2516
    assert(m->in_progress);
2517 2518 2519
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
2520
    if (!c) {
2521
      m->done = true;
2522 2523 2524 2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537
      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 已提交
2538
    }
2539 2540 2541 2542 2543 2544 2545 2546 2547 2548 2549 2550 2551 2552 2553 2554 2555 2556 2557 2558 2559 2560 2561 2562 2563 2564 2565 2566 2567 2568 2569 2570
  } else if (!compaction_queue_.empty()) {
    // cfd is referenced here
    auto cfd = PopFirstFromCompactionQueue();
    // We unreference here because the following code will take a Ref() on
    // this cfd if it is going to use it (Compaction class holds a
    // reference).
    // This will all happen under a mutex so we don't have to be afraid of
    // somebody else deleting it.
    if (cfd->Unref()) {
      delete cfd;
      // This was the last reference of the column family, so no need to
      // compact.
      return Status::OK();
    }

    // Pick up latest mutable CF Options and use it throughout the
    // compaction job
    // Compaction makes a copy of the latest MutableCFOptions. It should be used
    // throughout the compaction procedure to make sure consistency. It will
    // eventually be installed into SuperVersion
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    if (!mutable_cf_options->disable_auto_compactions && !cfd->IsDropped()) {
      // NOTE: try to avoid unnecessary copy of MutableCFOptions if
      // compaction is not necessary. Need to make sure mutex is held
      // until we make a copy in the following code
      c.reset(cfd->PickCompaction(*mutable_cf_options, log_buffer));
      if (c != nullptr) {
        // update statistics
        MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
                    c->inputs(0)->size());
        // There are three things that can change compaction score:
        // 1) When flush or compaction finish. This case is covered by
I
Igor Canadi 已提交
2571
        // InstallSuperVersionAndScheduleWork
2572
        // 2) When MutableCFOptions changes. This case is also covered by
I
Igor Canadi 已提交
2573 2574
        // InstallSuperVersionAndScheduleWork, because this is when the new
        // options take effect.
2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585
        // 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 已提交
2586
        }
I
Igor Canadi 已提交
2587 2588
      }
    }
J
jorlow@chromium.org 已提交
2589 2590
  }

2591
  if (!c) {
H
hans@chromium.org 已提交
2592
    // Nothing to do
2593
    LogToBuffer(log_buffer, "Compaction nothing to do");
2594
  } else if (c->deletion_compaction()) {
I
Igor Canadi 已提交
2595 2596 2597 2598
    // 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);
2599
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2600
           kCompactionStyleFIFO);
2601 2602 2603

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

I
Igor Canadi 已提交
2604
    for (const auto& f : *c->inputs(0)) {
2605
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2606
    }
2607 2608 2609
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
2610 2611
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2612 2613 2614 2615
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
    *madeProgress = true;
2616
  } else if (!trivial_move_disallowed && c->IsTrivialMove()) {
2617
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2618 2619 2620 2621 2622
    // Instrument for event update
    // TODO(yhchiang): add op details for showing trivial-move.
    ThreadStatusUtil::SetColumnFamily(c->column_family_data());
    ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);

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

2625 2626 2627
    // Move files to next level
    int32_t moved_files = 0;
    int64_t moved_bytes = 0;
2628
    for (unsigned int l = 0; l < c->num_input_levels(); l++) {
2629
      if (c->level(l) == c->output_level()) {
2630 2631 2632 2633
        continue;
      }
      for (size_t i = 0; i < c->num_input_files(l); i++) {
        FileMetaData* f = c->input(l, i);
2634
        c->edit()->DeleteFile(c->level(l), f->fd.GetNumber());
2635 2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646
        c->edit()->AddFile(c->output_level(), f->fd.GetNumber(),
                           f->fd.GetPathId(), f->fd.GetFileSize(), f->smallest,
                           f->largest, f->smallest_seqno, f->largest_seqno,
                           f->marked_for_compaction);

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

2649
    status = versions_->LogAndApply(c->column_family_data(),
2650 2651
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2652
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2653 2654
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
2655

S
sdong 已提交
2656
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Islam AbdelRahman 已提交
2657
    c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(),
2658
                                                             moved_bytes);
2659 2660 2661 2662
    {
      event_logger_.LogToBuffer(log_buffer)
          << "job" << job_context->job_id << "event"
          << "trivial_move"
I
Islam AbdelRahman 已提交
2663
          << "destination_level" << c->output_level() << "files" << moved_files
2664
          << "total_files_size" << moved_bytes;
2665
    }
2666 2667
    LogToBuffer(
        log_buffer,
2668
        "[%s] Moved #%d files to level-%d %" PRIu64 " bytes %s: %s\n",
I
Islam AbdelRahman 已提交
2669 2670
        c->column_family_data()->GetName().c_str(), moved_files,
        c->output_level(), moved_bytes, status.ToString().c_str(),
2671
        c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
2672
    *madeProgress = true;
2673 2674 2675

    // Clear Instrument
    ThreadStatusUtil::ResetThreadStatus();
J
jorlow@chromium.org 已提交
2676
  } else {
2677 2678 2679
    int output_level  __attribute__((unused)) = c->output_level();
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial",
                             &output_level);
I
Igor Canadi 已提交
2680
    assert(is_snapshot_supported_ || snapshots_.empty());
2681
    CompactionJob compaction_job(
I
Igor Canadi 已提交
2682 2683
        job_context->job_id, c.get(), db_options_, env_options_,
        versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(),
2684
        directories_.GetDataDir(c->output_path_id()), stats_,
2685
        snapshots_.GetAll(), table_cache_, &event_logger_,
2686 2687
        c->mutable_cf_options()->paranoid_file_checks,
        c->mutable_cf_options()->compaction_measure_io_stats, dbname_,
S
sdong 已提交
2688
        &compaction_job_stats);
I
Igor Canadi 已提交
2689
    compaction_job.Prepare();
2690

I
Igor Canadi 已提交
2691
    mutex_.Unlock();
2692
    compaction_job.Run();
2693
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun");
I
Igor Canadi 已提交
2694
    mutex_.Lock();
2695

2696
    status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
I
Igor Canadi 已提交
2697
    if (status.ok()) {
I
Igor Canadi 已提交
2698 2699
      InstallSuperVersionAndScheduleWorkWrapper(
          c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2700
    }
O
Ori Bernstein 已提交
2701 2702 2703
    *madeProgress = true;
  }
  if (c != nullptr) {
2704 2705 2706
    NotifyOnCompactionCompleted(
        c->column_family_data(), c.get(), status,
        compaction_job_stats, job_context->job_id);
I
Igor Canadi 已提交
2707
    c->ReleaseCompactionFiles(status);
2708
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2709
  }
2710
  // this will unref its input_version and column_family_data
2711
  c.reset();
J
jorlow@chromium.org 已提交
2712 2713 2714

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2715
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2716 2717
    // Ignore compaction errors found during shutting down
  } else {
2718
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2719
        status.ToString().c_str());
2720
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2721 2722 2723
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2724 2725

  if (is_manual) {
G
Gabor Cselle 已提交
2726
    ManualCompaction* m = manual_compaction_;
2727
    if (!status.ok()) {
L
Lei Jin 已提交
2728
      m->status = status;
2729 2730
      m->done = true;
    }
2731 2732 2733 2734 2735 2736 2737 2738 2739
    // 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.
2740 2741 2742 2743 2744
    //
    // 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) {
2745 2746
      m->done = true;
    }
G
Gabor Cselle 已提交
2747 2748 2749
    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 已提交
2750
      // Universal and FIFO compactions should always compact the whole range
S
sdong 已提交
2751 2752 2753
      assert(m->cfd->ioptions()->compaction_style !=
                 kCompactionStyleUniversal ||
             m->cfd->ioptions()->num_levels > 1);
2754
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2755
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2756 2757
      m->begin = &m->tmp_storage;
    }
2758
    m->in_progress = false; // not being processed anymore
2759
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2760
  }
2761
  return status;
J
jorlow@chromium.org 已提交
2762 2763
}

2764 2765
namespace {
struct IterState {
2766
  IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version)
I
Igor Canadi 已提交
2767
      : db(_db), mu(_mu), super_version(_super_version) {}
2768 2769

  DBImpl* db;
2770
  InstrumentedMutex* mu;
2771
  SuperVersion* super_version;
2772 2773 2774 2775
};

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

2777
  if (state->super_version->Unref()) {
2778 2779 2780
    // Job id == 0 means that this is not our background process, but rather
    // user thread
    JobContext job_context(0);
2781

2782 2783
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2784
    state->db->FindObsoleteFiles(&job_context, false, true);
2785 2786 2787
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2788 2789
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2790
    }
I
Igor Canadi 已提交
2791
    job_context.Clean();
I
Igor Canadi 已提交
2792
  }
T
Tomislav Novak 已提交
2793

2794 2795
  delete state;
}
H
Hans Wennborg 已提交
2796
}  // namespace
2797

L
Lei Jin 已提交
2798
Iterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
2799
                                      ColumnFamilyData* cfd,
2800 2801 2802
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
2803 2804 2805 2806 2807
  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 已提交
2808
      super_version->mem->NewIterator(read_options, arena));
2809
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
2810
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
2811
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
2812
  super_version->current->AddIterators(read_options, env_options_,
2813 2814
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
2815
  IterState* cleanup = new IterState(this, &mutex_, super_version);
2816
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2817 2818 2819 2820

  return internal_iter;
}

2821 2822 2823 2824
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2825
Status DBImpl::Get(const ReadOptions& read_options,
2826
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2827
                   std::string* value) {
L
Lei Jin 已提交
2828
  return GetImpl(read_options, column_family, key, value);
2829 2830
}

I
Igor Canadi 已提交
2831 2832
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
2833 2834
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
2835
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
2836
//
I
Igor Canadi 已提交
2837 2838 2839
// However, if InstallSuperVersionAndScheduleWork() gets called twice with the
// same job_context, we can't reuse the SuperVersion() that got
// malloced because
I
Igor Canadi 已提交
2840 2841 2842
// 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 已提交
2843
void DBImpl::InstallSuperVersionAndScheduleWorkWrapper(
I
Igor Canadi 已提交
2844
    ColumnFamilyData* cfd, JobContext* job_context,
2845
    const MutableCFOptions& mutable_cf_options) {
2846
  mutex_.AssertHeld();
I
Igor Canadi 已提交
2847
  SuperVersion* old_superversion = InstallSuperVersionAndScheduleWork(
I
Igor Canadi 已提交
2848 2849 2850
      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 已提交
2851 2852
}

I
Igor Canadi 已提交
2853
SuperVersion* DBImpl::InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
2854
    ColumnFamilyData* cfd, SuperVersion* new_sv,
I
Igor Canadi 已提交
2855
    const MutableCFOptions& mutable_cf_options) {
L
Lei Jin 已提交
2856
  mutex_.AssertHeld();
2857 2858 2859 2860 2861 2862 2863 2864 2865

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

2869
  // Whenever we install new SuperVersion, we might need to issue new flushes or
I
Igor Canadi 已提交
2870 2871 2872 2873
  // compactions.
  SchedulePendingFlush(cfd);
  SchedulePendingCompaction(cfd);
  MaybeScheduleFlushOrCompaction();
L
Lei Jin 已提交
2874 2875 2876 2877 2878 2879 2880 2881 2882 2883

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

2889 2890 2891
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2892
  SequenceNumber snapshot;
L
Lei Jin 已提交
2893 2894 2895
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2896 2897
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2898
  }
2899
  // Acquire SuperVersion
2900
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
2901
  // Prepare to store a list of merge operations if merge occurs.
2902
  MergeContext merge_context;
2903

2904
  Status s;
2905
  // First look in the memtable, then in the immutable memtable (if any).
2906
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2907
  // merge_operands will contain the sequence of merges in the latter case.
2908
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2909
  PERF_TIMER_STOP(get_snapshot_time);
2910

2911
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2912
    // Done
L
Lei Jin 已提交
2913
    RecordTick(stats_, MEMTABLE_HIT);
2914
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2915
    // Done
L
Lei Jin 已提交
2916
    RecordTick(stats_, MEMTABLE_HIT);
2917
  } else {
2918
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2919 2920
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2921
    RecordTick(stats_, MEMTABLE_MISS);
2922
  }
2923

2924 2925
  {
    PERF_TIMER_GUARD(get_post_process_time);
2926

2927
    ReturnAndCleanupSuperVersion(cfd, sv);
2928

2929 2930 2931
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2932
  return s;
J
jorlow@chromium.org 已提交
2933 2934
}

2935
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
2936
    const ReadOptions& read_options,
2937
    const std::vector<ColumnFamilyHandle*>& column_family,
2938
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
2939

L
Lei Jin 已提交
2940
  StopWatch sw(env_, stats_, DB_MULTIGET);
2941
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
2942

2943
  SequenceNumber snapshot;
2944

2945
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
2946
    ColumnFamilyData* cfd;
2947 2948 2949 2950 2951
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
2952 2953 2954 2955 2956 2957
    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});
2958 2959 2960
    }
  }

2961
  mutex_.Lock();
L
Lei Jin 已提交
2962 2963 2964
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2965 2966 2967
  } else {
    snapshot = versions_->LastSequence();
  }
2968
  for (auto mgd_iter : multiget_cf_data) {
2969 2970
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
2971
  }
2972
  mutex_.Unlock();
2973

2974 2975
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2976

2977
  // Note: this always resizes the values array
2978 2979 2980
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
2981 2982

  // Keep track of bytes that we read for statistics-recording later
2983
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
2984
  PERF_TIMER_STOP(get_snapshot_time);
2985 2986 2987 2988

  // 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.
2989
  // merge_operands will contain the sequence of merges in the latter case.
2990
  for (size_t i = 0; i < num_keys; ++i) {
2991
    merge_context.Clear();
2992
    Status& s = stat_list[i];
2993 2994 2995
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
2996 2997
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
2998 2999 3000
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
3001
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
3002
      // Done
3003
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
3004 3005
      // Done
    } else {
3006
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
3007 3008
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
3009 3010 3011
    }

    if (s.ok()) {
3012
      bytes_read += value->size();
3013 3014 3015 3016
    }
  }

  // Post processing (decrement reference counts and record statistics)
3017
  PERF_TIMER_GUARD(get_post_process_time);
3018 3019
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
3020
  // TODO(icanadi) do we need lock here or just around Cleanup()?
3021 3022 3023 3024 3025 3026
  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);
3027 3028
    }
  }
3029 3030 3031 3032 3033 3034 3035
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3036
  }
3037

L
Lei Jin 已提交
3038 3039 3040
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3041
  PERF_TIMER_STOP(get_post_process_time);
3042

3043
  return stat_list;
3044 3045
}

L
Lei Jin 已提交
3046
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3047
                                  const std::string& column_family_name,
3048
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
3049
  Status s;
I
Igor Canadi 已提交
3050
  *handle = nullptr;
3051 3052 3053 3054 3055 3056

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

Y
Yueh-Hsuan Chiang 已提交
3057
  {
3058
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
3059

Y
Yueh-Hsuan Chiang 已提交
3060 3061 3062 3063 3064 3065 3066 3067 3068 3069 3070 3071 3072 3073
    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 已提交
3074
    {  // write thread
3075 3076
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
I
Igor Canadi 已提交
3077 3078 3079 3080
      // LogAndApply will both write the creation in MANIFEST and create
      // ColumnFamilyData object
      s = versions_->LogAndApply(
          nullptr, MutableCFOptions(opt, ImmutableCFOptions(opt)), &edit,
3081
          &mutex_, directories_.GetDbDir(), false, &cf_options);
3082
      write_thread_.ExitUnbatched(&w);
I
Igor Canadi 已提交
3083
    }
Y
Yueh-Hsuan Chiang 已提交
3084 3085 3086 3087 3088
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
I
Igor Canadi 已提交
3089
      delete InstallSuperVersionAndScheduleWork(
Y
Yueh-Hsuan Chiang 已提交
3090
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
S
sdong 已提交
3091 3092 3093 3094 3095

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

Y
Yueh-Hsuan Chiang 已提交
3096 3097 3098 3099 3100 3101 3102 3103 3104
      *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());
    }
3105
  }  // InstrumentedMutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
3106 3107

  // this is outside the mutex
3108
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
3109 3110
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
3111
  }
3112
  return s;
3113 3114
}

3115 3116 3117 3118
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3119 3120
    return Status::InvalidArgument("Can't drop default column family");
  }
3121

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

I
Igor Canadi 已提交
3124 3125
  VersionEdit edit;
  edit.DropColumnFamily();
3126 3127
  edit.SetColumnFamily(cfd->GetID());

3128
  Status s;
3129
  {
3130
    InstrumentedMutexLock l(&mutex_);
3131 3132 3133 3134
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
3135
      // we drop column family from a single write thread
3136 3137
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
3138 3139
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
3140
      write_thread_.ExitUnbatched(&w);
3141
    }
S
sdong 已提交
3142 3143 3144 3145 3146 3147

    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()) {
3148
        if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) {
S
sdong 已提交
3149 3150 3151 3152 3153 3154
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
3155
  }
3156

3157
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
3158 3159 3160 3161
    // 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 已提交
3162
    assert(cfd->IsDropped());
L
Lei Jin 已提交
3163 3164 3165
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
3166 3167
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
3168
        cfd->GetID());
3169
  } else {
3170
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
3171
        "Dropping column family with id %u FAILED -- %s\n",
3172 3173 3174
        cfd->GetID(), s.ToString().c_str());
  }

3175
  return s;
3176 3177
}

L
Lei Jin 已提交
3178
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
3179 3180
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3181
  if (value_found != nullptr) {
K
Kai Liu 已提交
3182 3183
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3184
  }
L
Lei Jin 已提交
3185
  ReadOptions roptions = read_options;
3186
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3187
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3188

3189
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
3190 3191 3192
  // 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();
3193 3194
}

3195
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
3196 3197 3198
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3199

V
Venkatesh Radhakrishnan 已提交
3200 3201 3202 3203 3204 3205 3206 3207 3208 3209 3210 3211 3212 3213 3214 3215 3216 3217
  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 已提交
3218 3219 3220 3221
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
3222 3223
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
3224
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3225 3226 3227
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
3228
#endif
T
Tomislav Novak 已提交
3229
  } else {
3230
    SequenceNumber latest_snapshot = versions_->LastSequence();
3231
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
3232

I
Igor Canadi 已提交
3233
    auto snapshot =
3234 3235 3236
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
3237
            : latest_snapshot;
T
Tomislav Novak 已提交
3238

3239 3240 3241 3242 3243 3244 3245 3246 3247 3248 3249 3250 3251 3252 3253 3254 3255 3256 3257 3258 3259 3260 3261 3262 3263 3264 3265 3266 3267 3268 3269 3270 3271 3272 3273 3274 3275 3276 3277 3278 3279 3280 3281
    // 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(
3282
        env_, *cfd->ioptions(), cfd->user_comparator(),
3283
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
3284
        read_options.iterate_upper_bound);
3285

3286
    Iterator* internal_iter =
3287
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3288 3289 3290 3291
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
3292 3293
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
3294 3295
}

3296
Status DBImpl::NewIterators(
3297
    const ReadOptions& read_options,
I
Igor Canadi 已提交
3298
    const std::vector<ColumnFamilyHandle*>& column_families,
3299
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3300 3301
  iterators->clear();
  iterators->reserve(column_families.size());
V
Venkatesh Radhakrishnan 已提交
3302 3303 3304 3305 3306 3307 3308 3309 3310 3311 3312 3313 3314 3315 3316 3317 3318 3319 3320 3321
  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 已提交
3322 3323 3324 3325
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3326 3327
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
3328 3329
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
3330
      iterators->push_back(
3331
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3332 3333
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
3334
    }
I
Igor Canadi 已提交
3335
#endif
I
Igor Canadi 已提交
3336
  } else {
3337 3338
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
3339
    for (size_t i = 0; i < column_families.size(); ++i) {
3340 3341 3342
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
3343 3344

      auto snapshot =
3345 3346 3347
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
3348 3349
              : latest_snapshot;

3350
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3351
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
3352
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
3353
      Iterator* internal_iter = NewInternalIterator(
3354
          read_options, cfd, sv, db_iter->GetArena());
3355 3356
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3357 3358 3359 3360
    }
  }

  return Status::OK();
3361 3362
}

J
jorlow@chromium.org 已提交
3363
const Snapshot* DBImpl::GetSnapshot() {
3364 3365
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error
3366
  SnapshotImpl* s = new SnapshotImpl;
3367

3368
  InstrumentedMutexLock l(&mutex_);
3369
  // returns null if the underlying memtable does not support snapshot.
3370 3371 3372 3373 3374
  if (!is_snapshot_supported_) {
    delete s;
    return nullptr;
  }
  return snapshots_.New(s, versions_->LastSequence(), unix_time);
J
jorlow@chromium.org 已提交
3375 3376 3377
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
3378 3379 3380 3381 3382 3383
  const SnapshotImpl* casted_s = reinterpret_cast<const SnapshotImpl*>(s);
  {
    InstrumentedMutexLock l(&mutex_);
    snapshots_.Delete(casted_s);
  }
  delete casted_s;
J
jorlow@chromium.org 已提交
3384 3385 3386
}

// Convenience methods
3387 3388
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3389
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3390 3391
}

3392 3393 3394
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
3395
  if (!cfh->cfd()->ioptions()->merge_operator) {
3396 3397
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3398
    return DB::Merge(o, column_family, key, val);
3399 3400 3401
  }
}

L
Lei Jin 已提交
3402
Status DBImpl::Delete(const WriteOptions& write_options,
3403
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
3404
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
3405 3406
}

L
Lei Jin 已提交
3407
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
A
agiardullo 已提交
3408 3409 3410
  return WriteImpl(write_options, my_batch, nullptr);
}

A
agiardullo 已提交
3411
#ifndef ROCKSDB_LITE
A
agiardullo 已提交
3412 3413 3414 3415 3416
Status DBImpl::WriteWithCallback(const WriteOptions& write_options,
                                 WriteBatch* my_batch,
                                 WriteCallback* callback) {
  return WriteImpl(write_options, my_batch, callback);
}
A
agiardullo 已提交
3417
#endif  // ROCKSDB_LITE
A
agiardullo 已提交
3418 3419 3420

Status DBImpl::WriteImpl(const WriteOptions& write_options,
                         WriteBatch* my_batch, WriteCallback* callback) {
S
Stanislau Hlebik 已提交
3421 3422 3423
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
3424 3425 3426
  if (write_options.timeout_hint_us != 0) {
    return Status::InvalidArgument("timeout_hint_us is deprecated");
  }
A
agiardullo 已提交
3427 3428

  Status status;
A
agiardullo 已提交
3429 3430
  bool callback_failed = false;

A
agiardullo 已提交
3431 3432 3433 3434 3435 3436 3437 3438 3439 3440
  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;
  }

3441
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
3442
  WriteThread::Writer w;
S
Stanislau Hlebik 已提交
3443
  w.batch = my_batch;
L
Lei Jin 已提交
3444 3445
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
3446 3447
  w.in_batch_group = false;
  w.done = false;
A
agiardullo 已提交
3448
  w.has_callback = (callback != nullptr) ? true : false;
S
Stanislau Hlebik 已提交
3449

L
Lei Jin 已提交
3450
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
3451 3452 3453
    RecordTick(stats_, WRITE_WITH_WAL);
  }

3454 3455
  StopWatch write_sw(env_, db_options_.statistics.get(), DB_WRITE);

3456 3457 3458 3459 3460 3461 3462 3463
  write_thread_.JoinBatchGroup(&w);
  if (w.done) {
    // write was done by someone else, no need to grab mutex
    RecordTick(stats_, WRITE_DONE_BY_OTHER);
    return w.status;
  }
  // else we are the leader of the write batch group

S
Stanislau Hlebik 已提交
3464 3465
  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
3466 3467 3468 3469 3470

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

S
Stanislau Hlebik 已提交
3471 3472
  RecordTick(stats_, WRITE_DONE_BY_SELF);
  default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_SELF, 1);
3473

3474 3475 3476 3477
  // 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.
3478 3479 3480
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

3481
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
3482
                                    ? 4 * max_total_in_memory_state_
3483
                                    : db_options_.max_total_wal_size;
3484
  if (UNLIKELY(!single_column_family_mode_) &&
3485
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3486
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
3487
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
3488
    alive_log_files_.begin()->getting_flushed = true;
3489
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3490 3491 3492
        "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 已提交
3493 3494
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
3495
    for (auto cfd : *versions_->GetColumnFamilySet()) {
3496 3497 3498
      if (cfd->IsDropped()) {
        continue;
      }
I
Igor Canadi 已提交
3499
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
I
Igor Canadi 已提交
3500
        status = SwitchMemtable(cfd, &context);
I
Igor Canadi 已提交
3501 3502 3503
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
3504
        cfd->imm()->FlushRequested();
3505
        SchedulePendingFlush(cfd);
3506
      }
3507
    }
I
Igor Canadi 已提交
3508
    MaybeScheduleFlushOrCompaction();
3509 3510 3511 3512 3513 3514 3515 3516
  } 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()) {
3517 3518 3519
      if (cfd->IsDropped()) {
        continue;
      }
3520
      if (!cfd->mem()->IsEmpty()) {
I
Igor Canadi 已提交
3521
        status = SwitchMemtable(cfd, &context);
3522 3523 3524 3525
        if (!status.ok()) {
          break;
        }
        cfd->imm()->FlushRequested();
3526
        SchedulePendingFlush(cfd);
3527 3528 3529
      }
    }
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
3530 3531 3532 3533 3534 3535 3536 3537 3538 3539
  }

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

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

S
sdong 已提交
3540 3541
  if (UNLIKELY(status.ok()) &&
      (write_controller_.IsStopped() || write_controller_.NeedsDelay())) {
3542 3543
    PERF_TIMER_STOP(write_pre_and_post_process_time);
    PERF_TIMER_GUARD(write_delay_time);
S
sdong 已提交
3544 3545 3546 3547
    // We don't know size of curent batch so that we always use the size
    // for previous one. It might create a fairness issue that expiration
    // might happen for smaller writes but larger writes can go through.
    // Can optimize it if it is an issue.
3548
    status = DelayWrite(last_batch_group_size_);
3549
    PERF_TIMER_START(write_pre_and_post_process_time);
I
Igor Canadi 已提交
3550 3551
  }

D
dgrogan@chromium.org 已提交
3552
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
3553
  WriteThread::Writer* last_writer = &w;
A
agiardullo 已提交
3554
  autovector<WriteBatch*> write_batch_group;
3555 3556
  bool need_log_sync = !write_options.disableWAL && write_options.sync;
  bool need_log_dir_sync = need_log_sync && !log_dir_synced_;
A
agiardullo 已提交
3557

S
Stanislau Hlebik 已提交
3558
  if (status.ok()) {
3559 3560
    last_batch_group_size_ = write_thread_.EnterAsBatchGroupLeader(
        &w, &last_writer, &write_batch_group);
3561

3562
    if (need_log_sync) {
3563 3564 3565 3566 3567 3568 3569 3570 3571
      while (logs_.front().getting_synced) {
        log_sync_cv_.Wait();
      }
      for (auto& log : logs_) {
        assert(!log.getting_synced);
        log.getting_synced = true;
      }
    }

3572 3573 3574
    // 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
3575
    // into memtables
A
agiardullo 已提交
3576 3577 3578 3579 3580 3581 3582

    mutex_.Unlock();

    if (callback != nullptr) {
      // If this write has a validation callback, check to see if this write
      // is able to be written.  Must be called on the write thread.
      status = callback->Callback(this);
A
agiardullo 已提交
3583
      callback_failed = true;
A
agiardullo 已提交
3584 3585 3586 3587 3588 3589 3590 3591
    }
  } else {
    mutex_.Unlock();
  }

  // At this point the mutex is unlocked

  if (status.ok()) {
3592 3593 3594 3595 3596 3597 3598 3599 3600 3601
      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]);
        }
      }

3602 3603 3604 3605
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
3606
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
3607
      // Record statistics
L
Lei Jin 已提交
3608
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
3609
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
3610
      if (write_options.disableWAL) {
3611
        flush_on_destroy_ = true;
3612
      }
L
Lei Jin 已提交
3613
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3614

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

3652
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3653
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
3654
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
3655 3656 3657 3658 3659 3660 3661 3662
        // 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 已提交
3663
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
3664
      }
L
Lei Jin 已提交
3665
      PERF_TIMER_START(write_pre_and_post_process_time);
3666 3667 3668
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
3669
      mutex_.Lock();
A
agiardullo 已提交
3670

3671 3672 3673
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
3674 3675
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
3676
      if (!write_options.disableWAL) {
3677 3678 3679 3680 3681
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
3682
      if (status.ok()) {
3683
        versions_->SetLastSequence(last_sequence);
3684
      }
A
agiardullo 已提交
3685 3686 3687
  } else {
    // Operation failed.  Make sure sure mutex is held for cleanup code below.
    mutex_.Lock();
3688
  }
A
agiardullo 已提交
3689

A
agiardullo 已提交
3690
  if (db_options_.paranoid_checks && !status.ok() && !callback_failed &&
3691
      !status.IsBusy() && bg_error_.ok()) {
I
Igor Canadi 已提交
3692 3693
    bg_error_ = status; // stop compaction & fail any further writes
  }
3694

A
agiardullo 已提交
3695
  mutex_.AssertHeld();
3696

3697 3698
  if (need_log_sync) {
    MarkLogsSynced(logfile_number_, need_log_dir_sync, status);
3699 3700
  }

3701 3702 3703 3704 3705 3706 3707 3708 3709
  uint64_t writes_for_other = write_batch_group.size() - 1;
  if (writes_for_other > 0) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_OTHER,
                                           writes_for_other);
    if (!write_options.disableWAL) {
      default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_WITH_WAL,
                                             writes_for_other);
    }
  }
3710

I
Igor Canadi 已提交
3711
  mutex_.Unlock();
3712

3713 3714
  write_thread_.ExitAsBatchGroupLeader(&w, last_writer, status);

J
jorlow@chromium.org 已提交
3715 3716 3717
  return status;
}

3718 3719
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
3720
Status DBImpl::DelayWrite(uint64_t num_bytes) {
S
sdong 已提交
3721 3722 3723 3724
  uint64_t time_delayed = 0;
  bool delayed = false;
  {
    StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed);
S
sdong 已提交
3725 3726
    auto delay = write_controller_.GetDelay(env_, num_bytes);
    if (delay > 0) {
S
sdong 已提交
3727 3728
      mutex_.Unlock();
      delayed = true;
S
sdong 已提交
3729
      TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep");
3730 3731
      // hopefully we don't have to sleep more than 2 billion microseconds
      env_->SleepForMicroseconds(static_cast<int>(delay));
S
sdong 已提交
3732 3733
      mutex_.Lock();
    }
3734

3735
    while (bg_error_.ok() && write_controller_.IsStopped()) {
S
sdong 已提交
3736
      delayed = true;
3737 3738
      TEST_SYNC_POINT("DBImpl::DelayWrite:Wait");
      bg_cv_.Wait();
3739 3740
    }
  }
S
sdong 已提交
3741 3742 3743 3744 3745
  if (delayed) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_STALL_MICROS,
                                           time_delayed);
    RecordTick(stats_, STALL_MICROS, time_delayed);
  }
I
Igor Canadi 已提交
3746 3747

  return bg_error_;
3748 3749
}

I
Igor Canadi 已提交
3750 3751 3752
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
I
Igor Canadi 已提交
3753
    auto status = SwitchMemtable(cfd, context);
I
Igor Canadi 已提交
3754 3755 3756 3757 3758
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
3759 3760
    }
  }
I
Igor Canadi 已提交
3761
  return Status::OK();
S
Stanislau Hlebik 已提交
3762 3763 3764 3765
}

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
3766
Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
S
Stanislau Hlebik 已提交
3767 3768 3769 3770 3771 3772 3773
  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.
3774
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
3775 3776 3777 3778
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
3779
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
3780 3781 3782 3783
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
3784 3785
      EnvOptions opt_env_opt =
          env_->OptimizeForLogWrite(env_options_, db_options_);
3786
      s = env_->NewWritableFile(
3787
          LogFileName(db_options_.wal_dir, new_log_number), &lfile,
3788
          opt_env_opt);
S
Stanislau Hlebik 已提交
3789 3790 3791
      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 已提交
3792 3793
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
3794 3795 3796
        unique_ptr<WritableFileWriter> file_writer(
            new WritableFileWriter(std::move(lfile), opt_env_opt));
        new_log = new log::Writer(std::move(file_writer));
I
Igor Canadi 已提交
3797
      }
S
Stanislau Hlebik 已提交
3798 3799 3800
    }

    if (s.ok()) {
A
agiardullo 已提交
3801 3802
      SequenceNumber seq = versions_->LastSequence();
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options, seq);
S
Stanislau Hlebik 已提交
3803 3804 3805
      new_superversion = new SuperVersion();
    }
  }
3806
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
3807 3808
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
3809 3810 3811 3812 3813 3814 3815 3816 3817 3818 3819 3820
  mutex_.Lock();
  if (!s.ok()) {
    // how do we fail if we're not creating new log?
    assert(creating_new_log);
    assert(!new_mem);
    assert(!new_log);
    return s;
  }
  if (creating_new_log) {
    logfile_number_ = new_log_number;
    assert(new_log != nullptr);
    log_empty_ = true;
3821
    log_dir_synced_ = false;
3822
    logs_.emplace_back(logfile_number_, new_log);
S
Stanislau Hlebik 已提交
3823
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
3824
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
3825 3826 3827 3828
      // 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 已提交
3829
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
3830
          loop_cfd->imm()->NumNotFlushed() == 0) {
I
Igor Canadi 已提交
3831
        loop_cfd->SetLogNumber(logfile_number_);
3832
      }
3833 3834
    }
  }
S
Stanislau Hlebik 已提交
3835
  cfd->mem()->SetNextLogNumber(logfile_number_);
3836
  cfd->imm()->Add(cfd->mem(), &context->memtables_to_free_);
S
Stanislau Hlebik 已提交
3837 3838
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
I
Igor Canadi 已提交
3839 3840
  context->superversions_to_free_.push_back(InstallSuperVersionAndScheduleWork(
      cfd, new_superversion, mutable_cf_options));
3841 3842 3843
  return s;
}

I
Igor Canadi 已提交
3844
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
3845 3846 3847 3848 3849
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3850 3851
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
3852
  auto version = cfd->current();
3853 3854 3855 3856 3857 3858 3859 3860 3861 3862 3863 3864
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
3865
#endif  // ROCKSDB_LITE
3866

I
Igor Canadi 已提交
3867 3868 3869 3870
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3871 3872 3873 3874
Env* DBImpl::GetEnv() const {
  return env_;
}

3875 3876
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
3877
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
3878 3879
}

3880 3881
const DBOptions& DBImpl::GetDBOptions() const { return db_options_; }

3882
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
3883
                         const Slice& property, std::string* value) {
3884 3885
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3886 3887 3888
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

3889
  value->clear();
3890 3891 3892 3893 3894
  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) {
3895
      *value = ToString(int_value);
3896 3897 3898 3899 3900
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
3901
    InstrumentedMutexLock l(&mutex_);
3902 3903 3904
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
3905 3906
}

3907 3908
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
3909 3910
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3911 3912 3913 3914 3915 3916 3917 3918 3919 3920 3921 3922
  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) {
3923 3924
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
3925 3926

  if (!need_out_of_mutex) {
3927
    InstrumentedMutexLock l(&mutex_);
3928
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
3929 3930 3931 3932 3933 3934 3935 3936 3937 3938 3939 3940 3941 3942
  } 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 已提交
3943
  return cfd->GetThreadLocalSuperVersion(&mutex_);
3944 3945
}

A
agiardullo 已提交
3946 3947 3948 3949 3950 3951 3952 3953 3954 3955 3956 3957
// REQUIRED: this function should only be called on the write thread or if the
// mutex is held.
SuperVersion* DBImpl::GetAndRefSuperVersion(uint32_t column_family_id) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);
  if (!cfd) {
    return nullptr;
  }

  return GetAndRefSuperVersion(cfd);
}

A
agiardullo 已提交
3958 3959 3960 3961 3962 3963 3964 3965 3966 3967 3968 3969 3970 3971 3972 3973
// REQUIRED:  mutex is NOT held
SuperVersion* DBImpl::GetAndRefSuperVersionUnlocked(uint32_t column_family_id) {
  ColumnFamilyData* cfd;
  {
    InstrumentedMutexLock l(&mutex_);
    auto column_family_set = versions_->GetColumnFamilySet();
    cfd = column_family_set->GetColumnFamily(column_family_id);
  }

  if (!cfd) {
    return nullptr;
  }

  return GetAndRefSuperVersion(cfd);
}

3974 3975
void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
3976
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3977 3978 3979 3980 3981

  if (unref_sv) {
    // Release SuperVersion
    if (sv->Unref()) {
      {
3982
        InstrumentedMutexLock l(&mutex_);
3983 3984 3985 3986 3987 3988 3989
        sv->Cleanup();
      }
      delete sv;
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
    }
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
  }
3990 3991
}

A
agiardullo 已提交
3992 3993 3994 3995 3996 3997 3998 3999 4000 4001 4002 4003
// REQUIRED: this function should only be called on the write thread.
void DBImpl::ReturnAndCleanupSuperVersion(uint32_t column_family_id,
                                          SuperVersion* sv) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);

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

A
agiardullo 已提交
4004 4005 4006 4007 4008 4009 4010 4011 4012 4013 4014 4015 4016 4017 4018 4019
// REQUIRED: Mutex should NOT be held.
void DBImpl::ReturnAndCleanupSuperVersionUnlocked(uint32_t column_family_id,
                                                  SuperVersion* sv) {
  ColumnFamilyData* cfd;
  {
    InstrumentedMutexLock l(&mutex_);
    auto column_family_set = versions_->GetColumnFamilySet();
    cfd = column_family_set->GetColumnFamily(column_family_id);
  }

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

A
agiardullo 已提交
4020 4021 4022 4023 4024 4025 4026 4027 4028 4029 4030 4031
// REQUIRED: this function should only be called on the write thread or if the
// mutex is held.
ColumnFamilyHandle* DBImpl::GetColumnFamilyHandle(uint32_t column_family_id) {
  ColumnFamilyMemTables* cf_memtables = column_family_memtables_.get();

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

  return cf_memtables->GetColumnFamilyHandle();
}

A
agiardullo 已提交
4032 4033 4034 4035 4036 4037 4038 4039 4040 4041 4042 4043 4044 4045
// REQUIRED: mutex is NOT held.
ColumnFamilyHandle* DBImpl::GetColumnFamilyHandleUnlocked(
    uint32_t column_family_id) {
  ColumnFamilyMemTables* cf_memtables = column_family_memtables_.get();

  InstrumentedMutexLock l(&mutex_);

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

  return cf_memtables->GetColumnFamilyHandle();
}

4046
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4047 4048
                                 const Range* range, int n, uint64_t* sizes,
                                 bool include_memtable) {
J
jorlow@chromium.org 已提交
4049
  Version* v;
4050 4051
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4052 4053
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
  v = sv->current;
J
jorlow@chromium.org 已提交
4054 4055 4056

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
4057 4058 4059
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
    sizes[i] = versions_->ApproximateSize(v, k1.Encode(), k2.Encode());
4060 4061 4062 4063
    if (include_memtable) {
      sizes[i] += sv->mem->ApproximateSize(k1.Encode(), k2.Encode());
      sizes[i] += sv->imm->ApproximateSize(k1.Encode(), k2.Encode());
    }
J
jorlow@chromium.org 已提交
4064 4065
  }

4066
  ReturnAndCleanupSuperVersion(cfd, sv);
J
jorlow@chromium.org 已提交
4067 4068
}

I
Igor Canadi 已提交
4069 4070 4071 4072 4073 4074 4075 4076 4077 4078 4079 4080 4081 4082 4083 4084
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 已提交
4085 4086 4087 4088 4089
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
4090
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
4091 4092 4093
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
4094
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
4095 4096
}

4097 4098 4099
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4100 4101 4102
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
4103 4104
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
4105 4106 4107
    return Status::InvalidArgument("Invalid file name");
  }

4108 4109 4110 4111
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4112 4113
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
4114
          name.c_str());
4115 4116
      return Status::NotSupported("Delete only supported for archived logs");
    }
4117
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
4118
    if (!status.ok()) {
4119 4120
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
4121
          name.c_str(), status.ToString().c_str());
4122 4123 4124 4125
    }
    return status;
  }

4126
  int level;
I
Igor Canadi 已提交
4127
  FileMetaData* metadata;
4128
  ColumnFamilyData* cfd;
4129
  VersionEdit edit;
4130
  JobContext job_context(next_job_id_.fetch_add(1), true);
D
Dhruba Borthakur 已提交
4131
  {
4132
    InstrumentedMutexLock l(&mutex_);
4133
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4134
    if (!status.ok()) {
4135 4136
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
4137
      job_context.Clean();
D
Dhruba Borthakur 已提交
4138 4139
      return Status::InvalidArgument("File not found");
    }
4140
    assert(level < cfd->NumberLevels());
4141

D
Dhruba Borthakur 已提交
4142
    // If the file is being compacted no need to delete.
4143
    if (metadata->being_compacted) {
4144
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
4145
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
4146
      job_context.Clean();
D
Dhruba Borthakur 已提交
4147
      return Status::OK();
4148 4149
    }

D
Dhruba Borthakur 已提交
4150 4151 4152
    // 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 已提交
4153
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
4154
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
4155
      if (vstoreage->NumLevelFiles(i) != 0) {
4156
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
4157
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
4158
        job_context.Clean();
D
Dhruba Borthakur 已提交
4159 4160 4161
        return Status::InvalidArgument("File not in last level");
      }
    }
4162
    // if level == 0, it has to be the oldest file
S
sdong 已提交
4163 4164
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
4165 4166
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
4167
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
4168
      job_context.Clean();
4169 4170 4171
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
4172
    edit.DeleteFile(level, number);
4173
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
4174
                                    &edit, &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
4175
    if (status.ok()) {
I
Igor Canadi 已提交
4176 4177
      InstallSuperVersionAndScheduleWorkWrapper(
          cfd, &job_context, *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
4178
    }
I
Igor Canadi 已提交
4179 4180
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
4181

4182
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
4183
  // remove files outside the db-lock
I
Igor Canadi 已提交
4184
  if (job_context.HaveSomethingToDelete()) {
4185
    // Call PurgeObsoleteFiles() without holding mutex.
I
Igor Canadi 已提交
4186
    PurgeObsoleteFiles(job_context);
4187
  }
I
Igor Canadi 已提交
4188
  job_context.Clean();
4189 4190 4191
  return status;
}

4192
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
4193
  InstrumentedMutexLock l(&mutex_);
4194
  versions_->GetLiveFilesMetaData(metadata);
4195
}
4196 4197 4198 4199 4200 4201 4202 4203 4204 4205 4206

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 已提交
4207
#endif  // ROCKSDB_LITE
4208

I
Igor Canadi 已提交
4209 4210 4211 4212 4213 4214 4215
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
4219 4220 4221 4222 4223 4224
    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) {
4225
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
4226
                             ". Size recorded in manifest " +
4227 4228
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
4229 4230 4231 4232 4233 4234 4235 4236 4237
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

4238
Status DBImpl::GetDbIdentity(std::string& identity) const {
4239 4240
  std::string idfilename = IdentityFileName(dbname_);
  const EnvOptions soptions;
4241 4242 4243 4244 4245 4246 4247 4248 4249
  unique_ptr<SequentialFileReader> id_file_reader;
  Status s;
  {
    unique_ptr<SequentialFile> idfile;
    s = env_->NewSequentialFile(idfilename, &idfile, soptions);
    if (!s.ok()) {
      return s;
    }
    id_file_reader.reset(new SequentialFileReader(std::move(idfile)));
4250
  }
4251

4252 4253 4254 4255 4256
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
D
Dmitri Smirnov 已提交
4257
  char* buffer = reinterpret_cast<char*>(alloca(file_size));
4258
  Slice id;
4259
  s = id_file_reader->Read(static_cast<size_t>(file_size), &id, buffer);
4260 4261 4262 4263 4264 4265 4266 4267 4268 4269 4270
  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 已提交
4271 4272
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
4273
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
4274
               const Slice& key, const Slice& value) {
4275 4276 4277 4278
  // 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);
4279
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
4280 4281 4282
  return Write(opt, &batch);
}

4283 4284
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
4285
  WriteBatch batch;
4286
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
4287 4288 4289
  return Write(opt, &batch);
}

4290 4291
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
4292
  WriteBatch batch;
4293
  batch.Merge(column_family, key, value);
4294 4295 4296
  return Write(opt, &batch);
}

4297
// Default implementation -- returns not supported status
L
Lei Jin 已提交
4298
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
4299
                              const std::string& column_family_name,
4300
                              ColumnFamilyHandle** handle) {
4301
  return Status::NotSupported("");
4302
}
4303
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4304
  return Status::NotSupported("");
4305 4306
}

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

J
Jim Paton 已提交
4309
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4310 4311 4312 4313
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4314
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4315
  std::vector<ColumnFamilyHandle*> handles;
4316
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
4317 4318 4319 4320 4321 4322 4323
  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;
4324 4325
}

4326 4327
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4328
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4329
  Status s = SanitizeOptionsByTable(db_options, column_families);
4330 4331 4332
  if (!s.ok()) {
    return s;
  }
4333

4334 4335 4336 4337 4338 4339
  for (auto& cfd : column_families) {
    s = CheckCompressionSupported(cfd.options);
    if (!s.ok()) {
      return s;
    }
    if (db_options.db_paths.size() > 1) {
4340 4341
      if ((cfd.options.compaction_style != kCompactionStyleUniversal) &&
          (cfd.options.compaction_style != kCompactionStyleLevel)) {
4342 4343
        return Status::NotSupported(
            "More than one DB paths are only supported in "
4344
            "universal and level compaction styles. ");
4345 4346
      }
    }
4347
  }
4348

4349 4350 4351
  if (db_options.db_paths.size() > 4) {
    return Status::NotSupported(
        "More than four DB paths are not supported yet. ");
4352 4353
  }

4354
  *dbptr = nullptr;
4355
  handles->clear();
J
jorlow@chromium.org 已提交
4356

I
Igor Canadi 已提交
4357 4358 4359 4360
  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);
4361
  }
4362

I
Igor Canadi 已提交
4363
  DBImpl* impl = new DBImpl(db_options, dbname);
4364
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
4365
  if (s.ok()) {
4366
    for (auto db_path : impl->db_options_.db_paths) {
4367
      s = impl->env_->CreateDirIfMissing(db_path.path);
4368 4369 4370 4371 4372 4373
      if (!s.ok()) {
        break;
      }
    }
  }

4374 4375 4376 4377 4378 4379
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4380 4381 4382 4383
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4384
  impl->mutex_.Lock();
4385 4386
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4387
  if (s.ok()) {
4388
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4389
    unique_ptr<WritableFile> lfile;
4390
    EnvOptions soptions(db_options);
4391 4392
    EnvOptions opt_env_options =
        impl->db_options_.env->OptimizeForLogWrite(soptions, impl->db_options_);
4393 4394
    s = impl->db_options_.env->NewWritableFile(
        LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
4395
        opt_env_options);
J
jorlow@chromium.org 已提交
4396
    if (s.ok()) {
4397
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4398
      impl->logfile_number_ = new_log_number;
4399 4400
      unique_ptr<WritableFileWriter> file_writer(
          new WritableFileWriter(std::move(lfile), opt_env_options));
4401 4402
      impl->logs_.emplace_back(new_log_number,
                               new log::Writer(std::move(file_writer)));
I
Igor Canadi 已提交
4403

4404 4405
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4406 4407
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
4408 4409 4410
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
4411
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
4412 4413 4414 4415 4416 4417 4418 4419 4420 4421 4422 4423 4424 4425 4426 4427
        } 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 已提交
4428
        }
4429
      }
I
Igor Canadi 已提交
4430 4431
    }
    if (s.ok()) {
4432
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
4433
        delete impl->InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
4434
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
4435
      }
I
Igor Canadi 已提交
4436 4437
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4438
      impl->DeleteObsoleteFiles();
4439
      s = impl->directories_.GetDbDir()->Fsync();
J
jorlow@chromium.org 已提交
4440 4441
    }
  }
4442

I
Igor Canadi 已提交
4443 4444
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
S
sdong 已提交
4445
      if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
4446
        auto* vstorage = cfd->current()->storage_info();
4447
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
4448
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
4449
          if (num_files > 0) {
I
Igor Canadi 已提交
4450 4451
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
S
sdong 已提交
4452
                "open with FIFO compaction style.");
I
Igor Canadi 已提交
4453 4454 4455 4456
            break;
          }
        }
      }
S
sdong 已提交
4457 4458 4459
      if (!cfd->mem()->IsSnapshotSupported()) {
        impl->is_snapshot_supported_ = false;
      }
4460
      if (cfd->ioptions()->merge_operator != nullptr &&
4461 4462 4463 4464 4465
          !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 已提交
4466
      if (!s.ok()) {
4467 4468 4469 4470
        break;
      }
    }
  }
4471 4472 4473 4474 4475
  TEST_SYNC_POINT("DBImpl::Open:Opened");
  if (s.ok()) {
    impl->opened_successfully_ = true;
    impl->MaybeScheduleFlushOrCompaction();
  }
4476 4477
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4478
  if (s.ok()) {
S
sdong 已提交
4479 4480
    Log(InfoLogLevel::INFO_LEVEL, impl->db_options_.info_log, "DB pointer %p",
        impl);
J
jorlow@chromium.org 已提交
4481 4482
    *dbptr = impl;
  } else {
Y
Yueh-Hsuan Chiang 已提交
4483
    for (auto* h : *handles) {
4484 4485
      delete h;
    }
4486
    handles->clear();
J
jorlow@chromium.org 已提交
4487 4488 4489 4490 4491
    delete impl;
  }
  return s;
}

4492 4493 4494
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4495
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4496 4497
}

4498 4499 4500
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4501
Status DestroyDB(const std::string& dbname, const Options& options) {
4502
  const InternalKeyComparator comparator(options.comparator);
4503
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
4504
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4505
  std::vector<std::string> filenames;
4506

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

J
jorlow@chromium.org 已提交
4510
  FileLock* lock;
4511 4512
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4513 4514 4515
  if (result.ok()) {
    uint64_t number;
    FileType type;
4516
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
4517
    for (size_t i = 0; i < filenames.size(); i++) {
4518
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
4519
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4520 4521 4522 4523 4524 4525
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
4526 4527 4528 4529 4530
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4531

4532 4533
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
4534 4535 4536
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
4537
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
4538 4539 4540 4541 4542 4543 4544
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

I
Igor Canadi 已提交
4545 4546 4547 4548 4549 4550 4551 4552 4553 4554 4555 4556 4557 4558 4559 4560 4561 4562
    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;
4563
    env->GetChildren(archivedir, &archiveFiles);
4564 4565
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4566 4567
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4568
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4569 4570 4571 4572 4573
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4574
    // ignore case where no archival directory is present.
4575
    env->DeleteDir(archivedir);
4576

J
jorlow@chromium.org 已提交
4577
    env->UnlockFile(lock);  // Ignore error since state is already gone
4578
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4579
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4580
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4581 4582 4583 4584
  }
  return result;
}

Y
Yueh-Hsuan Chiang 已提交
4585
#if ROCKSDB_USING_THREAD_STATUS
4586

Y
Yueh-Hsuan Chiang 已提交
4587 4588
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4589
  if (db_options_.enable_thread_tracking) {
4590
    ThreadStatusUtil::NewColumnFamilyInfo(this, cfd);
4591
  }
Y
Yueh-Hsuan Chiang 已提交
4592 4593 4594 4595
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4596
  if (db_options_.enable_thread_tracking) {
4597
    ThreadStatusUtil::EraseColumnFamilyInfo(cfd);
4598
  }
Y
Yueh-Hsuan Chiang 已提交
4599 4600 4601
}

void DBImpl::EraseThreadStatusDbInfo() const {
4602
  if (db_options_.enable_thread_tracking) {
4603
    ThreadStatusUtil::EraseDatabaseInfo(this);
4604
  }
Y
Yueh-Hsuan Chiang 已提交
4605 4606 4607 4608 4609 4610 4611 4612 4613 4614 4615 4616 4617 4618 4619
}

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

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

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

4620 4621
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
4622
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4623
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
4624 4625
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
4626
  Warn(log,
4627
      "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
I
Igor Canadi 已提交
4628
      ROCKSDB_PATCH);
4629 4630
  Warn(log, "Git sha %s", rocksdb_build_git_sha);
  Warn(log, "Compile date %s", rocksdb_build_compile_date);
I
Igor Canadi 已提交
4631
#endif
4632 4633
}

A
agiardullo 已提交
4634 4635 4636 4637 4638 4639 4640 4641 4642 4643 4644 4645 4646 4647 4648 4649 4650 4651 4652 4653 4654 4655 4656 4657 4658 4659 4660 4661 4662 4663 4664 4665 4666 4667 4668 4669 4670 4671 4672 4673 4674 4675 4676 4677 4678 4679 4680 4681 4682 4683 4684 4685 4686 4687 4688 4689 4690 4691 4692 4693 4694 4695 4696 4697 4698 4699 4700 4701 4702 4703 4704 4705 4706 4707 4708 4709 4710 4711 4712
#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

4713
}  // namespace rocksdb