db_impl.cc 173.7 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>
17 18
#include <stdint.h>

J
jorlow@chromium.org 已提交
19
#include <algorithm>
20 21
#include <climits>
#include <cstdio>
J
jorlow@chromium.org 已提交
22
#include <set>
23
#include <stdexcept>
24
#include <string>
25
#include <unordered_map>
26
#include <unordered_set>
T
Tomislav Novak 已提交
27
#include <utility>
28
#include <vector>
29

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

94
namespace rocksdb {
J
jorlow@chromium.org 已提交
95

96
const std::string kDefaultColumnFamilyName("default");
97

I
Igor Canadi 已提交
98
void DumpRocksDBBuildVersion(Logger * log);
99

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

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

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

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

125 126
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
I
Igor Canadi 已提交
127 128 129 130 131
    int max_max_open_files = port::GetMaxOpenFiles();
    if (max_max_open_files == -1) {
      max_max_open_files = 1000000;
    }
    ClipToRange(&result.max_open_files, 20, max_max_open_files);
132
  }
133

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

I
Igor Canadi 已提交
147
  if (result.rate_limiter.get() != nullptr) {
148 149 150 151 152
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

153 154 155 156
  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
157
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
158
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
159
  }
160

161
  if (result.db_paths.size() == 0) {
162
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
163 164
  }

165 166 167 168
  if (result.compaction_readahead_size > 0) {
    result.new_table_reader_for_compaction_inputs = true;
  }

J
jorlow@chromium.org 已提交
169 170 171
  return result;
}

172 173
namespace {

174 175
Status SanitizeOptionsByTable(
    const DBOptions& db_opts,
176 177 178
    const std::vector<ColumnFamilyDescriptor>& column_families) {
  Status s;
  for (auto cf : column_families) {
179
    s = cf.options.table_factory->SanitizeOptions(db_opts, cf.options);
180 181 182 183 184 185 186
    if (!s.ok()) {
      return s;
    }
  }
  return Status::OK();
}

187
CompressionType GetCompressionFlush(const ImmutableCFOptions& ioptions) {
188 189 190 191 192 193
  // 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;

194
  if (ioptions.compaction_style == kCompactionStyleUniversal) {
195
    can_compress =
196
        (ioptions.compaction_options_universal.compression_size_percent < 0);
197 198
  } else {
    // For leveled compress when min_level_to_compress == 0.
199 200
    can_compress = ioptions.compression_per_level.empty() ||
                   ioptions.compression_per_level[0] != kNoCompression;
201 202 203
  }

  if (can_compress) {
204
    return ioptions.compression;
205 206 207 208
  } else {
    return kNoCompression;
  }
}
I
Igor Canadi 已提交
209

210
void DumpSupportInfo(Logger* logger) {
I
Igor Canadi 已提交
211 212 213 214 215 216 217 218
  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());
219 220
  Log(InfoLogLevel::INFO_LEVEL, logger, "Fast CRC32 supported: %d",
      crc32c::IsFastCrc32Supported());
I
Igor Canadi 已提交
221 222
}

223
}  // namespace
224

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

J
jorlow@chromium.org 已提交
268
  // Reserve ten files or so for other uses and give the rest to TableCache.
269
  // Give a large number for setting of "infinite" open files.
270 271
  const int table_cache_size = (db_options_.max_open_files == -1) ?
        4194304 : db_options_.max_open_files - 10;
272
  table_cache_ =
273
      NewLRUCache(table_cache_size, db_options_.table_cache_numshardbits);
274

275
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
276 277
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
I
Igor Canadi 已提交
278 279
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
280

I
Igor Canadi 已提交
281
  DumpRocksDBBuildVersion(db_options_.info_log.get());
282 283
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
284
  DumpSupportInfo(db_options_.info_log.get());
J
jorlow@chromium.org 已提交
285 286
}

287
// Will lock the mutex_,  will wait for completion if wait is true
288
void DBImpl::CancelAllBackgroundWork(bool wait) {
289
  InstrumentedMutexLock l(&mutex_);
290
  shutting_down_.store(true, std::memory_order_release);
291
  bg_cv_.SignalAll();
292 293 294 295
  if (!wait) {
    return;
  }
  // Wait for background work to finish
296
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
297 298
    bg_cv_.Wait();
  }
299 300
}

J
jorlow@chromium.org 已提交
301
DBImpl::~DBImpl() {
302
  mutex_.Lock();
303

304
  if (!shutting_down_.load(std::memory_order_acquire) && flush_on_destroy_) {
305
    for (auto cfd : *versions_->GetColumnFamilySet()) {
306
      if (!cfd->IsDropped() && !cfd->mem()->IsEmpty()) {
307 308 309 310
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
311
        cfd->Unref();
312 313
      }
    }
314
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
315
  }
316 317 318
  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
319 320
  // (to consider: moving all the waiting into CancelAllBackgroundWork(true))
  CancelAllBackgroundWork(false);
321 322 323 324 325 326 327
  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
328
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
329 330
    bg_cv_.Wait();
  }
331
  EraseThreadStatusDbInfo();
I
Igor Canadi 已提交
332 333
  flush_scheduler_.Clear();

334 335 336 337 338 339 340 341 342 343 344 345 346
  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 已提交
347 348 349 350 351
  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();
352 353
  }

I
Igor Canadi 已提交
354 355 356 357 358 359 360 361 362 363
  // 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_) {
364
    JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
365
    FindObsoleteFiles(&job_context, true);
366 367

    mutex_.Unlock();
I
Igor Canadi 已提交
368
    // manifest number starting from 2
I
Igor Canadi 已提交
369 370 371
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
372
    }
I
Igor Canadi 已提交
373
    job_context.Clean();
374
    mutex_.Lock();
375 376
  }

377 378 379
  for (auto l : logs_to_free_) {
    delete l;
  }
380 381 382
  for (auto& log : logs_) {
    log.ClearWriter();
  }
383
  logs_.clear();
384

385
  // versions need to be destroyed before table_cache since it can hold
386 387
  // references to table_cache.
  versions_.reset();
388
  mutex_.Unlock();
I
Igor Canadi 已提交
389 390 391
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
392

393
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
394 395 396
}

Status DBImpl::NewDB() {
397
  VersionEdit new_db;
398
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
399 400 401
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

402 403
  Status s;

404 405
  Log(InfoLogLevel::INFO_LEVEL,
      db_options_.info_log, "Creating manifest 1 \n");
J
jorlow@chromium.org 已提交
406 407
  const std::string manifest = DescriptorFileName(dbname_, 1);
  {
408 409 410 411 412 413 414 415 416 417
    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 已提交
418 419 420
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
S
sdong 已提交
421 422 423
    if (s.ok()) {
      s = SyncManifest(env_, &db_options_, log.file());
    }
J
jorlow@chromium.org 已提交
424 425 426
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
427
    s = SetCurrentFile(env_, dbname_, 1, directories_.GetDbDir());
J
jorlow@chromium.org 已提交
428 429 430 431 432 433 434
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
435
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
436 437
    // No change needed
  } else {
438 439
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
440 441 442 443
    *s = Status::OK();
  }
}

444
const Status DBImpl::CreateArchivalDirectory() {
445 446
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
447 448 449 450 451
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

452
void DBImpl::PrintStatistics() {
453
  auto dbstats = db_options_.statistics.get();
454
  if (dbstats) {
Y
Yueh-Hsuan Chiang 已提交
455
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
M
Mark Callaghan 已提交
456
        "STATISTICS:\n %s",
457
        dbstats->ToString().c_str());
458 459 460
  }
}

461
void DBImpl::MaybeDumpStats() {
462
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
463 464 465 466

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
467
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
468 469 470 471 472 473
      <= 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;
474

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

500
    PrintStatistics();
501 502 503
  }
}

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

517
  // if deletion is disabled, do nothing
518
  if (disable_delete_obsolete_files_ > 0) {
519 520 521
    return;
  }

522 523 524 525 526
  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;
527
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
528 529 530
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
I
Igor Canadi 已提交
531
    if (delete_obsolete_files_next_run_ < now_micros) {
532
      doing_the_full_scan = true;
I
Igor Canadi 已提交
533 534
      delete_obsolete_files_next_run_ =
          now_micros + db_options_.delete_obsolete_files_period_micros;
535 536 537
    }
  }

I
Igor Canadi 已提交
538 539
  // don't delete files that might be currently written to from compaction
  // threads
540 541 542
  // Since job_context->min_pending_output is set, until file scan finishes,
  // mutex_ cannot be released. Otherwise, we might see no min_pending_output
  // here but later find newer generated unfinalized files while scannint.
I
Igor Canadi 已提交
543 544 545 546 547 548 549
  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();
  }

550 551
  // Get obsolete files.  This function will also update the list of
  // pending files in VersionSet().
I
Igor Canadi 已提交
552 553
  versions_->GetObsoleteFiles(&job_context->sst_delete_files,
                              job_context->min_pending_output);
I
Igor Canadi 已提交
554

I
Igor Canadi 已提交
555
  // store the current filenum, lognum, etc
556
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
557
  job_context->pending_manifest_file_number =
558
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
559
  job_context->log_number = versions_->MinLogNumber();
560
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
561

I
Igor Canadi 已提交
562
  versions_->AddLiveFiles(&job_context->sst_live);
563
  if (doing_the_full_scan) {
I
Igor Canadi 已提交
564 565
    for (uint32_t path_id = 0; path_id < db_options_.db_paths.size();
         path_id++) {
566 567 568
      // set of all files in the directory. We'll exclude files that are still
      // alive in the subsequent processings.
      std::vector<std::string> files;
569
      env_->GetChildren(db_options_.db_paths[path_id].path,
570
                        &files);  // Ignore errors
571
      for (std::string file : files) {
I
Igor Canadi 已提交
572
        // TODO(icanadi) clean up this mess to avoid having one-off "/" prefixes
I
Igor Canadi 已提交
573 574
        job_context->full_scan_candidate_files.emplace_back("/" + file,
                                                            path_id);
575 576
      }
    }
577 578

    //Add log files in wal_dir
579
    if (db_options_.wal_dir != dbname_) {
580
      std::vector<std::string> log_files;
581
      env_->GetChildren(db_options_.wal_dir, &log_files);  // Ignore errors
582
      for (std::string log_file : log_files) {
I
Igor Canadi 已提交
583
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
584 585
      }
    }
586
    // Add info log files in db_log_dir
587
    if (!db_options_.db_log_dir.empty() && db_options_.db_log_dir != dbname_) {
588
      std::vector<std::string> info_log_files;
589 590
      // Ignore errors
      env_->GetChildren(db_options_.db_log_dir, &info_log_files);
591
      for (std::string log_file : info_log_files) {
I
Igor Canadi 已提交
592
        job_context->full_scan_candidate_files.emplace_back(log_file, 0);
593 594
      }
    }
595
  }
596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626

  if (!alive_log_files_.empty()) {
    uint64_t min_log_number = versions_->MinLogNumber();
    // 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());
    }
    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;
      }
      logs_to_free_.push_back(log.ReleaseWriter());
      logs_.pop_front();
    }
    // Current log cannot be obsolete.
    assert(!logs_.empty());
  }

  // We're just cleaning up for DB::Write().
  assert(job_context->logs_to_free.empty());
  job_context->logs_to_free = logs_to_free_;
  logs_to_free_.clear();
627 628
}

629
namespace {
I
Igor Canadi 已提交
630 631
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
632 633 634 635 636
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
637
    return (first.path_id > second.path_id);
638 639 640 641
  }
}
};  // namespace

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

I
Igor Canadi 已提交
650 651 652 653
  // 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 已提交
654 655
    return;
  }
656

657
  // Now, convert live list to an unordered map, WITHOUT mutex held;
658
  // set is slow.
659
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
I
Igor Canadi 已提交
660
  for (const FileDescriptor& fd : state.sst_live) {
661 662
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
663

I
Igor Canadi 已提交
664
  auto candidate_files = state.full_scan_candidate_files;
I
Igor Canadi 已提交
665 666 667
  candidate_files.reserve(candidate_files.size() +
                          state.sst_delete_files.size() +
                          state.log_delete_files.size());
K
kailiu 已提交
668 669
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
670
  for (auto file : state.sst_delete_files) {
671 672 673
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
674
    delete file;
I
Igor Canadi 已提交
675 676
  }

K
kailiu 已提交
677 678
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
679 680
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
681 682
    }
  }
683

K
kailiu 已提交
684
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
685
  // file twice
686
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
687 688
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
689

690
  std::vector<std::string> old_info_log_files;
691
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
692 693 694
  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 已提交
695 696 697
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
698
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
699 700
      continue;
    }
J
jorlow@chromium.org 已提交
701

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

    if (keep) {
      continue;
    }

746
    std::string fname;
K
kailiu 已提交
747 748
    if (type == kTableFile) {
      // evict from cache
749
      TableCache::Evict(table_cache_.get(), number);
750
      fname = TableFileName(db_options_.db_paths, number, path_id);
751
    } else {
752 753
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
754
    }
755

756
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
757
    if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 ||
758
                              db_options_.WAL_size_limit_MB > 0)) {
I
Igor Canadi 已提交
759
      wal_manager_.ArchiveWALFile(fname, number);
760 761 762
      continue;
    }
#endif  // !ROCKSDB_LITE
I
Islam AbdelRahman 已提交
763
    Status file_deletion_status;
764 765
    if (type == kTableFile && path_id == 0) {
      file_deletion_status = DeleteOrMoveToTrash(&db_options_, fname);
I
Islam AbdelRahman 已提交
766 767 768
    } else {
      file_deletion_status = env_->DeleteFile(fname);
    }
769
    if (file_deletion_status.ok()) {
770
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
771
          "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", state.job_id,
772 773
          fname.c_str(), type, number,
          file_deletion_status.ToString().c_str());
774 775 776 777 778 779
    } else if (env_->FileExists(fname).IsNotFound()) {
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
          "[JOB %d] Tried to delete a non-existing file %s type=%d #%" PRIu64
          " -- %s\n",
          state.job_id, fname.c_str(), type, number,
          file_deletion_status.ToString().c_str());
780 781 782 783 784 785 786
    } 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) {
787 788 789 790
      EventHelpers::LogAndNotifyTableFileDeletion(
          &event_logger_, state.job_id, number, fname,
          file_deletion_status, GetName(),
          db_options_.listeners);
J
jorlow@chromium.org 已提交
791 792
    }
  }
H
heyongqiang 已提交
793

794
  // Delete old info log files.
795
  size_t old_info_log_file_count = old_info_log_files.size();
796
  if (old_info_log_file_count >= db_options_.keep_log_file_num) {
797
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
798
    size_t end = old_info_log_file_count - db_options_.keep_log_file_num;
799
    for (unsigned int i = 0; i <= end; i++) {
800
      std::string& to_delete = old_info_log_files.at(i);
801 802
      std::string full_path_to_delete = (db_options_.db_log_dir.empty() ?
           dbname_ : db_options_.db_log_dir) + "/" + to_delete;
803
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
804
          "[JOB %d] Delete info log file %s\n", state.job_id,
805 806
          full_path_to_delete.c_str());
      Status s = env_->DeleteFile(full_path_to_delete);
807
      if (!s.ok()) {
808 809 810 811 812 813 814 815 816 817
        if (env_->FileExists(full_path_to_delete).IsNotFound()) {
          Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
              "[JOB %d] Tried to delete non-existing info log file %s FAILED "
              "-- %s\n",
              state.job_id, to_delete.c_str(), s.ToString().c_str());
        } else {
          Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
              "[JOB %d] Delete info log file %s FAILED -- %s\n", state.job_id,
              to_delete.c_str(), s.ToString().c_str());
        }
818
      }
H
heyongqiang 已提交
819 820
    }
  }
I
Igor Canadi 已提交
821 822 823
#ifndef ROCKSDB_LITE
  wal_manager_.PurgeObsoleteWALFiles();
#endif  // ROCKSDB_LITE
824
  LogFlush(db_options_.info_log);
D
Dhruba Borthakur 已提交
825 826 827 828
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
829
  JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
830
  FindObsoleteFiles(&job_context, true);
831 832

  mutex_.Unlock();
I
Igor Canadi 已提交
833 834
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
835
  }
I
Igor Canadi 已提交
836
  job_context.Clean();
837
  mutex_.Lock();
838 839
}

840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855
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 已提交
856

857 858 859 860 861 862 863 864 865
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_);
866 867 868
    if (!s.ok()) {
      return s;
    }
869
  }
870

871 872 873 874 875 876 877 878
  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);
879 880 881
      if (!s.ok()) {
        return s;
      }
882
      data_dirs_.emplace_back(path_directory.release());
883
    }
884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902
  }
  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();
903

904 905 906 907 908
  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);
909 910 911 912
    if (!s.ok()) {
      return s;
    }

913
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
914 915 916
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
917

A
agiardullo 已提交
918 919
    s = env_->FileExists(CurrentFileName(dbname_));
    if (s.IsNotFound()) {
920
      if (db_options_.create_if_missing) {
921
        s = NewDB();
922
        is_new_db = true;
923 924 925 926 927 928
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
929
      }
A
agiardullo 已提交
930
    } else if (s.ok()) {
931
      if (db_options_.error_if_exists) {
932 933 934
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
A
agiardullo 已提交
935 936 937 938
    } else {
      // Unexpected error reading file
      assert(s.IsIOError());
      return s;
J
jorlow@chromium.org 已提交
939
    }
M
Mayank Agarwal 已提交
940
    // Check for the IDENTITY file and create it if not there
A
agiardullo 已提交
941 942
    s = env_->FileExists(IdentityFileName(dbname_));
    if (s.IsNotFound()) {
M
Mayank Agarwal 已提交
943 944 945 946
      s = SetIdentityFile(env_, dbname_);
      if (!s.ok()) {
        return s;
      }
A
agiardullo 已提交
947 948 949
    } else if (!s.ok()) {
      assert(s.IsIOError());
      return s;
M
Mayank Agarwal 已提交
950
    }
J
jorlow@chromium.org 已提交
951 952
  }

953
  Status s = versions_->Recover(column_families, read_only);
954
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
955 956
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
957
  if (s.ok()) {
A
agiardullo 已提交
958
    SequenceNumber max_sequence(kMaxSequenceNumber);
959 960
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
961
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
962 963
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
964 965 966 967 968

    // 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).
    //
969
    // Note that prev_log_number() is no longer used, but we pay
970
    // attention to it in case we are recovering a database
971
    // produced by an older version of rocksdb.
972
    const uint64_t min_log = versions_->MinLogNumber();
973
    const uint64_t prev_log = versions_->prev_log_number();
974
    std::vector<std::string> filenames;
975
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
976 977
    if (!s.ok()) {
      return s;
978
    }
K
kailiu 已提交
979

980 981
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
982 983
      uint64_t number;
      FileType type;
984 985 986 987 988 989 990 991 992
      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);
        }
993
      }
J
jorlow@chromium.org 已提交
994
    }
995

H
heyongqiang 已提交
996 997 998 999 1000 1001
    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 已提交
1002 1003 1004 1005 1006 1007 1008
    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 已提交
1009 1010
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 kMaxSequenceNumber);
S
Stanislau Hlebik 已提交
1011 1012
        }
      }
1013
    }
L
Lei Jin 已提交
1014
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
1015 1016
  }

L
Lei Jin 已提交
1017 1018
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
1019
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
1020 1021 1022
    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 已提交
1023 1024
  }

J
jorlow@chromium.org 已提交
1025 1026 1027
  return s;
}

S
Stanislau Hlebik 已提交
1028 1029 1030
// 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 已提交
1031 1032
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
1033
    Logger* info_log;
J
jorlow@chromium.org 已提交
1034
    const char* fname;
1035
    Status* status;  // nullptr if db_options_.paranoid_checks==false
I
Igor Sugak 已提交
1036
    virtual void Corruption(size_t bytes, const Status& s) override {
1037 1038
      Log(InfoLogLevel::WARN_LEVEL,
          info_log, "%s%s: dropping %d bytes; %s",
1039
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
1040
          fname, static_cast<int>(bytes), s.ToString().c_str());
K
krad 已提交
1041 1042 1043
      if (this->status != nullptr && this->status->ok()) {
        *this->status = s;
      }
J
jorlow@chromium.org 已提交
1044 1045 1046 1047
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
1048
  Status status;
1049
  std::unordered_map<int, VersionEdit> version_edits;
1050
  // no need to refcount because iteration is under mutex
1051 1052
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1053 1054
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1055
  }
1056 1057 1058 1059 1060 1061 1062 1063 1064 1065 1066 1067
  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 已提交
1068

K
krad 已提交
1069
  bool continue_replay_log = true;
S
Stanislau Hlebik 已提交
1070 1071 1072 1073
  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.
1074
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
1075 1076
    // Open the log file
    std::string fname = LogFileName(db_options_.wal_dir, log_number);
1077 1078 1079 1080
    unique_ptr<SequentialFileReader> file_reader;
    {
      unique_ptr<SequentialFile> file;
      status = env_->NewSequentialFile(fname, &file, env_options_);
S
Stanislau Hlebik 已提交
1081
      if (!status.ok()) {
1082 1083 1084 1085 1086 1087 1088 1089
        MaybeIgnoreError(&status);
        if (!status.ok()) {
          return status;
        } else {
          // Fail with one log file, but that's ok.
          // Try next one.
          continue;
        }
S
Stanislau Hlebik 已提交
1090
      }
1091
      file_reader.reset(new SequentialFileReader(std::move(file)));
J
jorlow@chromium.org 已提交
1092 1093
    }

S
Stanislau Hlebik 已提交
1094 1095 1096 1097 1098
    // Create the log reader.
    LogReporter reporter;
    reporter.env = env_;
    reporter.info_log = db_options_.info_log.get();
    reporter.fname = fname.c_str();
K
krad 已提交
1099 1100 1101 1102 1103 1104 1105
    if (!db_options_.paranoid_checks ||
        db_options_.wal_recovery_mode ==
            WALRecoveryMode::kSkipAnyCorruptedRecords) {
      reporter.status = nullptr;
    } else {
      reporter.status = &status;
    }
S
Stanislau Hlebik 已提交
1106 1107 1108 1109
    // 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).
1110
    log::Reader reader(std::move(file_reader), &reporter, true /*checksum*/,
S
Stanislau Hlebik 已提交
1111
                       0 /*initial_offset*/);
K
krad 已提交
1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122 1123 1124 1125 1126 1127 1128 1129 1130
    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 已提交
1131 1132 1133 1134 1135

    // Read all the records and add to a memtable
    std::string scratch;
    Slice record;
    WriteBatch batch;
K
krad 已提交
1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148

    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 已提交
1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163 1164 1165
      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()) {
1166 1167 1168 1169
        // We are treating this as a failure while reading since we read valid
        // blocks that do not form coherent data
        reporter.Corruption(record.size(), status);
        continue;
S
Stanislau Hlebik 已提交
1170
      }
1171

S
Stanislau Hlebik 已提交
1172 1173
      const SequenceNumber last_seq = WriteBatchInternal::Sequence(&batch) +
                                      WriteBatchInternal::Count(&batch) - 1;
A
agiardullo 已提交
1174
      if ((*max_sequence == kMaxSequenceNumber) || (last_seq > *max_sequence)) {
S
Stanislau Hlebik 已提交
1175 1176 1177 1178
        *max_sequence = last_seq;
      }

      if (!read_only) {
I
Igor Canadi 已提交
1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190
        // 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;
1191
          status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
I
Igor Canadi 已提交
1192 1193 1194 1195
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
1196
          }
A
agiardullo 已提交
1197 1198 1199

          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 *max_sequence);
1200
        }
J
jorlow@chromium.org 已提交
1201 1202 1203
      }
    }

1204
    if (!status.ok()) {
K
krad 已提交
1205
      if (db_options_.wal_recovery_mode ==
1206 1207 1208 1209
             WALRecoveryMode::kSkipAnyCorruptedRecords) {
        // We should ignore all errors unconditionally
        status = Status::OK();
      } else if (db_options_.wal_recovery_mode ==
K
krad 已提交
1210 1211 1212 1213 1214 1215 1216 1217
                 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);
1218 1219 1220 1221 1222
      } else {
        assert(db_options_.wal_recovery_mode ==
                  WALRecoveryMode::kTolerateCorruptedTailRecords
               || db_options_.wal_recovery_mode ==
                  WALRecoveryMode::kAbsoluteConsistency);
K
krad 已提交
1223 1224
        return status;
      }
1225 1226
    }

I
Igor Canadi 已提交
1227
    flush_scheduler_.Clear();
A
agiardullo 已提交
1228 1229
    if ((*max_sequence != kMaxSequenceNumber) &&
        (versions_->LastSequence() < *max_sequence)) {
S
Stanislau Hlebik 已提交
1230 1231
      versions_->SetLastSequence(*max_sequence);
    }
1232 1233
  }

1234
  if (!read_only) {
1235 1236
    // 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 已提交
1237
    auto max_log_number = log_numbers.back();
1238
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1239
      auto iter = version_edits.find(cfd->GetID());
1240 1241 1242
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
1243
      if (cfd->GetLogNumber() > max_log_number) {
1244
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
1245
        // from all logs. Memtable has to be empty because
1246
        // we filter the updates based on log_number
1247
        // (in WriteBatch::InsertInto)
1248 1249 1250 1251 1252 1253 1254
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1255
        status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
1256 1257 1258 1259
        if (!status.ok()) {
          // Recovery failed
          break;
        }
A
agiardullo 已提交
1260 1261 1262

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

1265
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
1266
      // writing log_number in the manifest means that any log file
1267 1268
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
1269 1270 1271
      // 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);
1272 1273 1274
      // 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 已提交
1275
      // log number
1276
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
1277 1278
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
1279
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1280 1281
        // Recovery failed
        break;
1282 1283
      }
    }
1284
  }
I
Igor Canadi 已提交
1285

1286 1287 1288
  event_logger_.Log() << "job" << job_id << "event"
                      << "recovery_finished";

J
jorlow@chromium.org 已提交
1289 1290 1291
  return status;
}

1292 1293
Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
                                           MemTable* mem, VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1294
  mutex_.AssertHeld();
1295
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1296
  FileMetaData meta;
1297
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1298 1299
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1300 1301
  ReadOptions ro;
  ro.total_order_seek = true;
1302
  Arena arena;
1303
  Status s;
1304
  TableProperties table_properties;
1305
  {
1306
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
1307 1308 1309
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1310
        cfd->GetName().c_str(), meta.fd.GetNumber());
1311

1312 1313
    bool paranoid_file_checks =
        cfd->GetLatestMutableCFOptions()->paranoid_file_checks;
1314 1315
    {
      mutex_.Unlock();
1316
      TableFileCreationInfo info;
1317 1318
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
1319
          iter.get(), &meta, cfd->internal_comparator(),
1320 1321
          cfd->int_tbl_prop_collector_factories(), cfd->GetID(),
          snapshots_.GetAll(), GetCompressionFlush(*cfd->ioptions()),
1322 1323
          cfd->ioptions()->compression_opts, paranoid_file_checks,
          cfd->internal_stats(), Env::IO_HIGH, &info.table_properties);
1324
      LogFlush(db_options_.info_log);
1325 1326 1327 1328 1329 1330 1331 1332
      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()) {
1333 1334 1335 1336 1337 1338 1339 1340 1341
        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);
1342
      }
1343 1344
      mutex_.Lock();
    }
1345
  }
I
Igor Canadi 已提交
1346
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1347 1348 1349 1350

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1351
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1352 1353
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
1354 1355
                  meta.smallest_seqno, meta.largest_seqno,
                  meta.marked_for_compaction);
1356 1357
  }

L
Lei Jin 已提交
1358
  InternalStats::CompactionStats stats(1);
1359
  stats.micros = env_->NowMicros() - start_micros;
1360
  stats.bytes_written = meta.fd.GetFileSize();
1361
  stats.num_output_files = 1;
1362
  cfd->internal_stats()->AddCompactionStats(level, stats);
1363 1364
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1365
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1366 1367 1368
  return s;
}

1369 1370
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
1371
    bool* made_progress, JobContext* job_context, LogBuffer* log_buffer) {
1372
  mutex_.AssertHeld();
1373
  assert(cfd->imm()->NumNotFlushed() != 0);
1374
  assert(cfd->imm()->IsFlushPending());
1375

I
Igor Canadi 已提交
1376 1377
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1378
                     snapshots_.GetAll(), job_context, log_buffer,
1379
                     directories_.GetDbDir(), directories_.GetDataDir(0U),
I
Igor Canadi 已提交
1380 1381
                     GetCompressionFlush(*cfd->ioptions()), stats_,
                     &event_logger_);
1382

1383
  FileMetaData file_meta;
1384 1385 1386 1387 1388 1389 1390

  // 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.
1391
  Status s = flush_job.Run(&file_meta);
J
jorlow@chromium.org 已提交
1392 1393

  if (s.ok()) {
I
Igor Canadi 已提交
1394 1395
    InstallSuperVersionAndScheduleWorkWrapper(cfd, job_context,
                                              mutable_cf_options);
1396 1397
    if (made_progress) {
      *made_progress = 1;
1398
    }
S
sdong 已提交
1399
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1400
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1401
                cfd->current()->storage_info()->LevelSummary(&tmp));
J
jorlow@chromium.org 已提交
1402
  }
1403

1404
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1405 1406 1407 1408 1409
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1410
  RecordFlushIOStats();
1411 1412 1413
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1414
    NotifyOnFlushCompleted(cfd, &file_meta, mutable_cf_options,
1415
                           job_context->job_id, flush_job.GetTableProperties());
1416 1417
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1418 1419 1420
  return s;
}

1421 1422 1423 1424
void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd,
                                    FileMetaData* file_meta,
                                    const MutableCFOptions& mutable_cf_options,
                                    int job_id, TableProperties prop) {
1425
#ifndef ROCKSDB_LITE
1426
  if (db_options_.listeners.size() == 0U) {
1427 1428
    return;
  }
1429 1430 1431 1432
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
1433
  bool triggered_writes_slowdown =
1434
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1435
       mutable_cf_options.level0_slowdown_writes_trigger);
1436
  bool triggered_writes_stop =
1437
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1438
       mutable_cf_options.level0_stop_writes_trigger);
1439 1440
  // release lock while notifying events
  mutex_.Unlock();
1441
  {
1442 1443 1444 1445 1446
    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,
1447
                                       file_meta->fd.GetNumber());
1448
    info.thread_id = env_->GetThreadID();
1449 1450 1451
    info.job_id = job_id;
    info.triggered_writes_slowdown = triggered_writes_slowdown;
    info.triggered_writes_stop = triggered_writes_stop;
1452 1453
    info.smallest_seqno = file_meta->smallest_seqno;
    info.largest_seqno = file_meta->largest_seqno;
1454
    info.table_properties = prop;
1455
    for (auto listener : db_options_.listeners) {
1456
      listener->OnFlushCompleted(this, info);
1457 1458
    }
  }
1459 1460 1461
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
1462
#endif  // ROCKSDB_LITE
1463 1464
}

1465 1466 1467 1468
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()) {
1469 1470 1471
    return Status::InvalidArgument("Invalid target path ID");
  }

1472 1473
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1474 1475

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1476
  if (!s.ok()) {
1477
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1478 1479 1480
    return s;
  }

I
Igor Canadi 已提交
1481
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1482
  {
1483
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
1484
    Version* base = cfd->current();
1485 1486
    for (int level = 1; level < base->storage_info()->num_non_empty_levels();
         level++) {
S
sdong 已提交
1487
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1488 1489 1490 1491
        max_level_with_files = level;
      }
    }
  }
1492

1493
  int final_output_level = 0;
1494 1495 1496 1497
  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
1498 1499
                            cfd->NumberLevels() - 1, options.target_path_id,
                            begin, end);
1500
    final_output_level = cfd->NumberLevels() - 1;
1501 1502
  } else {
    for (int level = 0; level <= max_level_with_files; level++) {
1503
      int output_level;
A
Andres Notzli 已提交
1504
      // in case the compaction is universal or if we're compacting the
1505 1506 1507 1508
      // 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 ||
1509 1510 1511
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
        output_level = level;
      } else if (level == max_level_with_files && level > 0) {
1512 1513 1514 1515 1516 1517 1518
        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 已提交
1519
                   cfd->ioptions()->compaction_filter_factory == nullptr) {
A
Andres Notzli 已提交
1520 1521
          // Skip bottommost level compaction since we don't have a compaction
          // filter
1522 1523
          continue;
        }
1524
        output_level = level;
1525
      } else {
1526
        output_level = level + 1;
1527 1528 1529 1530 1531
        if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
            cfd->ioptions()->level_compaction_dynamic_level_bytes &&
            level == 0) {
          output_level = ColumnFamilyData::kCompactToBaseLevel;
        }
1532
      }
1533 1534
      s = RunManualCompaction(cfd, level, output_level, options.target_path_id,
                              begin, end);
1535 1536 1537
      if (!s.ok()) {
        break;
      }
1538 1539 1540 1541 1542
      if (output_level == ColumnFamilyData::kCompactToBaseLevel) {
        final_output_level = cfd->NumberLevels() - 1;
      } else if (output_level > final_output_level) {
        final_output_level = output_level;
      }
1543 1544
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
1545
    }
G
Gabor Cselle 已提交
1546
  }
1547 1548 1549 1550
  if (!s.ok()) {
    LogFlush(db_options_.info_log);
    return s;
  }
1551

1552
  if (options.change_level) {
1553 1554 1555 1556 1557 1558 1559
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop");
    s = PauseBackgroundWork();
    if (s.ok()) {
      s = ReFitLevel(cfd, final_output_level, options.target_level);
    }
    ContinueBackgroundWork();
1560
  }
1561
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1562

1563
  {
1564
    InstrumentedMutexLock l(&mutex_);
1565 1566 1567 1568 1569
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1570
  return s;
1571 1572
}

1573 1574 1575 1576 1577
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 已提交
1578 1579 1580 1581
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1582 1583 1584 1585 1586 1587
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);
1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612

  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());
1613
  }  // release the mutex
1614 1615 1616 1617 1618 1619 1620 1621 1622 1623

  // 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()) {
1624
      // no mutex is locked here.  No need to Unlock() and Lock() here.
1625 1626 1627 1628 1629
      PurgeObsoleteFiles(job_context);
    }
    job_context.Clean();
  }

1630
  return s;
I
Igor Canadi 已提交
1631
#endif  // ROCKSDB_LITE
1632 1633
}

I
Igor Canadi 已提交
1634
#ifndef ROCKSDB_LITE
1635 1636 1637
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
1638 1639
    const int output_level, int output_path_id, JobContext* job_context,
    LogBuffer* log_buffer) {
1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651 1652 1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671
  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;
  }

1672
  std::vector<CompactionInputFiles> input_files;
1673 1674 1675 1676 1677 1678 1679 1680 1681 1682 1683 1684 1685 1686 1687 1688 1689 1690 1691 1692
  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(
1693 1694
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
1695 1696 1697
  assert(c);
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
1698
  assert(!c->deletion_compaction());
1699

I
Igor Canadi 已提交
1700
  assert(is_snapshot_supported_ || snapshots_.empty());
1701
  CompactionJob compaction_job(
I
Igor Canadi 已提交
1702 1703
      job_context->job_id, c.get(), db_options_, env_options_, versions_.get(),
      &shutting_down_, log_buffer, directories_.GetDbDir(),
S
sdong 已提交
1704
      directories_.GetDataDir(c->output_path_id()), stats_, snapshots_.GetAll(),
1705
      table_cache_, &event_logger_,
1706 1707
      c->mutable_cf_options()->paranoid_file_checks,
      c->mutable_cf_options()->compaction_measure_io_stats, dbname_,
1708 1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719 1720
      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.
1721 1722 1723
  compaction_job.Prepare();

  mutex_.Unlock();
1724
  compaction_job.Run();
1725
  mutex_.Lock();
1726

1727
  Status status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
1728
  if (status.ok()) {
I
Igor Canadi 已提交
1729 1730
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
1731 1732 1733 1734 1735 1736 1737 1738 1739
  }
  c->ReleaseCompactionFiles(s);
  c.reset();

  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
1740 1741
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] [JOB %d] Compaction error: %s",
1742
        c->column_family_data()->GetName().c_str(), job_context->job_id,
1743 1744 1745 1746 1747 1748 1749
        status.ToString().c_str());
    if (db_options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }

  bg_compaction_scheduled_--;
1750 1751 1752
  if (bg_compaction_scheduled_ == 0) {
    bg_cv_.SignalAll();
  }
1753 1754 1755

  return status;
}
I
Igor Canadi 已提交
1756
#endif  // ROCKSDB_LITE
1757

1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768 1769 1770 1771 1772 1773 1774 1775 1776
Status DBImpl::PauseBackgroundWork() {
  InstrumentedMutexLock guard_lock(&mutex_);
  bg_work_paused_++;
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_ > 0) {
    bg_cv_.Wait();
  }
  return Status::OK();
}

Status DBImpl::ContinueBackgroundWork() {
  InstrumentedMutexLock guard_lock(&mutex_);
  assert(bg_work_paused_ > 0);
  bg_work_paused_--;
  if (bg_work_paused_ == 0) {
    MaybeScheduleFlushOrCompaction();
  }
  return Status::OK();
}

O
Ori Bernstein 已提交
1777
void DBImpl::NotifyOnCompactionCompleted(
1778 1779
    ColumnFamilyData* cfd, Compaction *c, const Status &st,
    const CompactionJobStats& compaction_job_stats,
1780
    const int job_id) {
O
Ori Bernstein 已提交
1781
#ifndef ROCKSDB_LITE
1782
  if (db_options_.listeners.size() == 0U) {
O
Ori Bernstein 已提交
1783 1784 1785 1786 1787 1788 1789 1790
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  // release lock while notifying events
  mutex_.Unlock();
1791 1792 1793 1794
  {
    CompactionJobInfo info;
    info.cf_name = cfd->GetName();
    info.status = st;
1795
    info.thread_id = env_->GetThreadID();
1796 1797
    info.job_id = job_id;
    info.base_input_level = c->start_level();
1798
    info.output_level = c->output_level();
1799
    info.stats = compaction_job_stats;
1800
    info.table_properties = c->GetOutputTableProperties();
1801 1802
    for (size_t i = 0; i < c->num_input_levels(); ++i) {
      for (const auto fmd : *c->inputs(i)) {
1803 1804 1805 1806 1807 1808 1809 1810 1811 1812 1813
        auto fn = TableFileName(db_options_.db_paths, fmd->fd.GetNumber(),
                                fmd->fd.GetPathId());
        info.input_files.push_back(fn);
        if (info.table_properties.count(fn) == 0) {
          std::shared_ptr<const TableProperties> tp;
          std::string fname;
          auto s = cfd->current()->GetTableProperties(&tp, fmd, &fname);
          if (s.ok()) {
            info.table_properties[fn] = tp;
          }
        }
1814 1815 1816 1817 1818 1819 1820 1821 1822 1823 1824 1825
      }
    }
    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 已提交
1826 1827 1828 1829 1830 1831
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
#endif  // ROCKSDB_LITE
}

1832
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1833
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1834 1835 1836
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1837 1838
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1839 1840
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1841
        cfd->GetName().c_str());
1842
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1843 1844 1845
  }

  MutableCFOptions new_options;
1846
  Status s;
L
Lei Jin 已提交
1847
  {
1848
    InstrumentedMutexLock l(&mutex_);
1849 1850
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1851 1852 1853 1854
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

1855 1856
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1857 1858
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1859 1860
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1861
  }
1862
  if (s.ok()) {
1863 1864
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1865 1866 1867
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
  } else {
1868 1869
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1870
  }
S
sdong 已提交
1871
  LogFlush(db_options_.info_log);
1872
  return s;
I
Igor Canadi 已提交
1873
#endif  // ROCKSDB_LITE
1874 1875
}

1876
// return the same level if it cannot be moved
1877 1878
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
1879
  mutex_.AssertHeld();
S
sdong 已提交
1880
  const auto* vstorage = cfd->current()->storage_info();
1881
  int minimum_level = level;
1882
  for (int i = level - 1; i > 0; --i) {
1883
    // stop if level i is not empty
S
sdong 已提交
1884
    if (vstorage->NumLevelFiles(i) > 0) break;
1885
    // stop if level i is too small (cannot fit the level files)
1886
    if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) {
1887 1888
      break;
    }
1889 1890 1891 1892 1893 1894

    minimum_level = i;
  }
  return minimum_level;
}

1895 1896
// REQUIREMENT: block all background work by calling PauseBackgroundWork()
// before calling this function
I
Igor Canadi 已提交
1897 1898
Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
  assert(level < cfd->NumberLevels());
1899 1900 1901
  if (target_level >= cfd->NumberLevels()) {
    return Status::InvalidArgument("Target level exceeds number of levels");
  }
1902

1903 1904 1905 1906
  std::unique_ptr<SuperVersion> superversion_to_free;
  std::unique_ptr<SuperVersion> new_superversion(new SuperVersion());

  Status status;
I
Igor Canadi 已提交
1907

I
Islam AbdelRahman 已提交
1908
  InstrumentedMutexLock guard_lock(&mutex_);
1909 1910 1911

  // only allow one thread refitting
  if (refitting_level_) {
1912 1913
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
L
Lei Jin 已提交
1914
    return Status::NotSupported("another thread is refitting");
1915 1916 1917
  }
  refitting_level_ = true;

1918
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
1919
  // move to a smaller level
1920 1921
  int to_level = target_level;
  if (target_level < 0) {
1922
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1923
  }
1924

1925 1926 1927 1928 1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939
  auto* vstorage = cfd->current()->storage_info();
  if (to_level > level) {
    if (level == 0) {
      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) {
        return Status::NotSupported(
            "Levels between source and target are not empty for a move.");
      }
    }
  }
  if (to_level != level) {
1940
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
1941 1942
        "[%s] Before refitting:\n%s", cfd->GetName().c_str(),
        cfd->current()->DebugString().data());
1943

1944
    VersionEdit edit;
I
Igor Canadi 已提交
1945
    edit.SetColumnFamily(cfd->GetID());
1946
    for (const auto& f : vstorage->LevelFiles(level)) {
1947
      edit.DeleteFile(level, f->fd.GetNumber());
1948 1949
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
1950 1951
                   f->smallest_seqno, f->largest_seqno,
                   f->marked_for_compaction);
1952
    }
1953
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
1954 1955
        "[%s] Apply version edit:\n%s", cfd->GetName().c_str(),
        edit.DebugString().data());
1956

1957 1958
    status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                    directories_.GetDbDir());
1959 1960
    superversion_to_free.reset(InstallSuperVersionAndScheduleWork(
        cfd, new_superversion.release(), mutable_cf_options));
1961

1962 1963
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1964
        status.ToString().data());
1965 1966

    if (status.ok()) {
1967
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
1968 1969
          "[%s] After refitting:\n%s", cfd->GetName().c_str(),
          cfd->current()->DebugString().data());
1970 1971 1972 1973
    }
  }

  refitting_level_ = false;
I
Igor Canadi 已提交
1974

L
Lei Jin 已提交
1975
  return status;
G
Gabor Cselle 已提交
1976 1977
}

1978 1979 1980
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1981 1982
}

1983
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
1984
  return 0;
1985 1986
}

1987 1988
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1989
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
1990 1991
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
1992 1993
}

L
Lei Jin 已提交
1994
Status DBImpl::Flush(const FlushOptions& flush_options,
1995 1996
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1997
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
1998 1999
}

2000 2001 2002 2003 2004 2005 2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025 2026 2027 2028 2029 2030 2031
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;
2032
      logs_to_sync.push_back(log.writer);
2033 2034 2035 2036 2037
    }

    need_log_dir_sync = !log_dir_synced_;
  }

2038
  RecordTick(stats_, WAL_FILE_SYNCED);
2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051 2052 2053 2054 2055 2056 2057 2058 2059 2060 2061 2062 2063 2064 2065 2066 2067 2068 2069
  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) {
2070
      logs_to_free_.push_back(log.ReleaseWriter());
A
Andres Noetzli 已提交
2071
      it = logs_.erase(it);
2072 2073 2074 2075 2076
    } else {
      log.getting_synced = false;
      ++it;
    }
  }
A
Andres Noetzli 已提交
2077
  assert(logs_.empty() || (logs_.size() == 1 && !logs_[0].getting_synced));
2078 2079 2080
  log_sync_cv_.SignalAll();
}

2081
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
2082 2083 2084
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
2085
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
2086
                                   int output_level, uint32_t output_path_id,
2087 2088
                                   const Slice* begin, const Slice* end,
                                   bool disallow_trivial_move) {
2089 2090
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);
2091

G
Gabor Cselle 已提交
2092 2093
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
2094
  ManualCompaction manual;
I
Igor Canadi 已提交
2095
  manual.cfd = cfd;
2096 2097
  manual.input_level = input_level;
  manual.output_level = output_level;
2098
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
2099
  manual.done = false;
2100
  manual.in_progress = false;
2101
  manual.disallow_trivial_move = disallow_trivial_move;
2102 2103 2104
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
2105 2106
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2107
    manual.begin = nullptr;
G
Gabor Cselle 已提交
2108
  } else {
2109
    begin_storage.SetMaxPossibleForUserKey(*begin);
G
Gabor Cselle 已提交
2110 2111
    manual.begin = &begin_storage;
  }
2112
  if (end == nullptr ||
2113 2114
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2115
    manual.end = nullptr;
G
Gabor Cselle 已提交
2116
  } else {
2117
    end_storage.SetMinPossibleForUserKey(*end);
G
Gabor Cselle 已提交
2118 2119 2120
    manual.end = &end_storage;
  }

2121
  InstrumentedMutexLock l(&mutex_);
2122

2123 2124 2125 2126 2127 2128
  // 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
2129
  // RunManualCompaction(), i.e. during that time no other compaction will
2130 2131 2132
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
2133
  // RunManualCompaction() from getting to the second while loop below.
2134 2135 2136 2137 2138
  // 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) {
2139
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
2140 2141 2142
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
2143 2144
    bg_cv_.Wait();
  }
2145

2146 2147
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
2148
      cfd->GetName().c_str());
2149

2150 2151 2152 2153
  // 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) {
2154 2155 2156
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
2157
      bg_cv_.Wait();
2158 2159
    } else {
      manual_compaction_ = &manual;
2160
      bg_compaction_scheduled_++;
2161
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
G
Gabor Cselle 已提交
2162
    }
H
hans@chromium.org 已提交
2163
  }
2164

2165 2166 2167
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
2168
  return manual.status;
J
jorlow@chromium.org 已提交
2169 2170
}

2171 2172 2173 2174 2175 2176 2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187
InternalIterator* DBImpl::NewInternalIterator(
    Arena* arena, ColumnFamilyHandle* column_family) {
  ColumnFamilyData* cfd;
  if (column_family == nullptr) {
    cfd = default_cf_handle_->cfd();
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    cfd = cfh->cfd();
  }

  mutex_.Lock();
  SuperVersion* super_version = cfd->GetSuperVersion()->Ref();
  mutex_.Unlock();
  ReadOptions roptions;
  return NewInternalIterator(roptions, cfd, super_version, arena);
}

2188
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
2189
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
2190 2191 2192
  Status s;
  {
    WriteContext context;
2193
    InstrumentedMutexLock guard_lock(&mutex_);
2194

2195
    if (cfd->imm()->NumNotFlushed() == 0 && cfd->mem()->IsEmpty()) {
2196 2197 2198 2199
      // Nothing to flush
      return Status::OK();
    }

2200 2201
    WriteThread::Writer w;
    write_thread_.EnterUnbatched(&w, &mutex_);
S
Stanislau Hlebik 已提交
2202

I
Igor Canadi 已提交
2203
    // SwitchMemtable() will release and reacquire mutex
S
Stanislau Hlebik 已提交
2204
    // during execution
I
Igor Canadi 已提交
2205
    s = SwitchMemtable(cfd, &context);
2206
    write_thread_.ExitUnbatched(&w);
2207

S
Stanislau Hlebik 已提交
2208 2209
    cfd->imm()->FlushRequested();

2210 2211 2212
    // schedule flush
    SchedulePendingFlush(cfd);
    MaybeScheduleFlushOrCompaction();
S
Stanislau Hlebik 已提交
2213
  }
S
Stanislau Hlebik 已提交
2214

L
Lei Jin 已提交
2215
  if (s.ok() && flush_options.wait) {
2216
    // Wait until the compaction completes
2217
    s = WaitForFlushMemTable(cfd);
2218 2219
  }
  return s;
J
jorlow@chromium.org 已提交
2220 2221
}

2222
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
2223 2224
  Status s;
  // Wait until the compaction completes
2225
  InstrumentedMutexLock l(&mutex_);
2226
  while (cfd->imm()->NumNotFlushed() > 0 && bg_error_.ok()) {
2227 2228 2229
    if (shutting_down_.load(std::memory_order_acquire)) {
      return Status::ShutdownInProgress();
    }
2230 2231
    bg_cv_.Wait();
  }
2232
  if (!bg_error_.ok()) {
2233 2234 2235
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
2236 2237
}

2238
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
2239
  mutex_.AssertHeld();
2240 2241 2242 2243
  if (!opened_successfully_) {
    // Compaction may introduce data race to DB open
    return;
  }
2244 2245
  if (bg_work_paused_ > 0) {
    // we paused the background work
2246
    return;
I
Igor Canadi 已提交
2247
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
2248
    // DB is being deleted; no more background compactions
2249 2250
    return;
  }
2251

2252 2253 2254 2255
  while (unscheduled_flushes_ > 0 &&
         bg_flush_scheduled_ < db_options_.max_background_flushes) {
    unscheduled_flushes_--;
    bg_flush_scheduled_++;
2256
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
2257 2258
  }

2259 2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270
  // 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);
    }
  }

2271 2272 2273 2274 2275 2276
  if (bg_manual_only_) {
    // only manual compactions are allowed to run. don't schedule automatic
    // compactions
    return;
  }

2277 2278 2279 2280
  while (bg_compaction_scheduled_ < db_options_.max_background_compactions &&
         unscheduled_compactions_ > 0) {
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
2281
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
2282 2283 2284 2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303 2304 2305 2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326 2327
  }
}

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_;
2328 2329 2330
  }
}

2331
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
2332
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
2333 2334 2335
  IOSTATS_RESET(bytes_written);
}

2336
void DBImpl::BGWorkFlush(void* db) {
2337
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
2338
  TEST_SYNC_POINT("DBImpl::BGWorkFlush");
2339
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
2340
  TEST_SYNC_POINT("DBImpl::BGWorkFlush:done");
2341 2342 2343
}

void DBImpl::BGWorkCompaction(void* db) {
2344
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
2345
  TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
2346 2347 2348
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

2349
Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
H
Haobo Xu 已提交
2350
                               LogBuffer* log_buffer) {
2351
  mutex_.AssertHeld();
2352

2353 2354 2355 2356 2357 2358 2359
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
    return status;
2360 2361
  }

2362 2363 2364
  ColumnFamilyData* cfd = nullptr;
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
I
Igor Canadi 已提交
2365
    auto first_cfd = PopFirstFromFlushQueue();
2366

I
Igor Canadi 已提交
2367
    if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
2368
      // can't flush this CF, try next one
I
Igor Canadi 已提交
2369 2370
      if (first_cfd->Unref()) {
        delete first_cfd;
2371 2372
      }
      continue;
2373
    }
2374 2375

    // found a flush!
I
Igor Canadi 已提交
2376
    cfd = first_cfd;
2377 2378 2379 2380 2381 2382 2383 2384 2385 2386 2387 2388 2389
    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_);
2390
    status = FlushMemTableToOutputFile(cfd, mutable_cf_options, made_progress,
2391 2392 2393
                                       job_context, log_buffer);
    if (cfd->Unref()) {
      delete cfd;
2394
    }
J
jorlow@chromium.org 已提交
2395
  }
2396
  return status;
J
jorlow@chromium.org 已提交
2397 2398
}

2399
void DBImpl::BackgroundCallFlush() {
2400
  bool made_progress = false;
2401
  JobContext job_context(next_job_id_.fetch_add(1), true);
2402 2403
  assert(bg_flush_scheduled_);

2404
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2405
  {
2406
    InstrumentedMutexLock l(&mutex_);
H
Haobo Xu 已提交
2407

I
Igor Canadi 已提交
2408 2409 2410
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2411
    Status s = BackgroundFlush(&made_progress, &job_context, &log_buffer);
2412 2413 2414 2415 2416 2417 2418 2419 2420 2421 2422 2423 2424 2425 2426 2427 2428
    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 已提交
2429 2430
    }

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

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

H
Haobo Xu 已提交
2452
    bg_flush_scheduled_--;
2453 2454
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2455
    RecordFlushIOStats();
H
Haobo Xu 已提交
2456
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2457 2458 2459 2460
    // 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.
2461
  }
J
jorlow@chromium.org 已提交
2462 2463
}

2464
void DBImpl::BackgroundCallCompaction() {
2465
  bool made_progress = false;
2466
  JobContext job_context(next_job_id_.fetch_add(1), true);
H
Haobo Xu 已提交
2467 2468

  MaybeDumpStats();
2469
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2470
  {
2471
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
2472 2473 2474 2475

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2476
    assert(bg_compaction_scheduled_);
2477
    Status s = BackgroundCompaction(&made_progress, &job_context, &log_buffer);
2478 2479 2480 2481 2482 2483
    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 =
2484
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2485 2486 2487 2488 2489 2490 2491 2492 2493 2494
      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();
2495
    }
H
Haobo Xu 已提交
2496

I
Igor Canadi 已提交
2497 2498
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2499 2500 2501 2502
    // 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());
2503 2504

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2505
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2506
      mutex_.Unlock();
2507 2508 2509 2510 2511
      // 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 已提交
2512
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2513 2514
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2515
      }
I
Igor Canadi 已提交
2516
      job_context.Clean();
2517 2518
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2519

2520
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2521

2522 2523
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2524 2525
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2526
    if (made_progress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
2527
      // signal if
2528
      // * made_progress -- need to wakeup DelayWrite
2529 2530 2531 2532 2533 2534
      // * 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 已提交
2535 2536 2537 2538
    // 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.
2539
  }
J
jorlow@chromium.org 已提交
2540 2541
}

2542 2543
Status DBImpl::BackgroundCompaction(bool* made_progress,
                                    JobContext* job_context,
2544
                                    LogBuffer* log_buffer) {
2545
  *made_progress = false;
J
jorlow@chromium.org 已提交
2546
  mutex_.AssertHeld();
2547

2548 2549
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2550 2551
  bool trivial_move_disallowed = is_manual &&
                                 manual_compaction_->disallow_trivial_move;
2552

2553
  CompactionJobStats compaction_job_stats;
2554 2555 2556 2557 2558 2559
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
2560
    if (is_manual) {
2561
      manual_compaction_->status = status;
2562 2563 2564 2565
      manual_compaction_->done = true;
      manual_compaction_->in_progress = false;
      manual_compaction_ = nullptr;
    }
2566
    return status;
2567 2568
  }

2569 2570 2571
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2572 2573 2574 2575
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2576 2577 2578
  }

  unique_ptr<Compaction> c;
2579 2580
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2581
  if (is_manual) {
G
Gabor Cselle 已提交
2582
    ManualCompaction* m = manual_compaction_;
2583
    assert(m->in_progress);
2584 2585 2586
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
2587
    if (!c) {
2588
      m->done = true;
2589 2590 2591 2592 2593 2594 2595 2596 2597 2598 2599 2600 2601 2602 2603 2604
      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 已提交
2605
    }
2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628 2629 2630 2631 2632 2633 2634 2635 2636 2637
  } 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 已提交
2638
        // InstallSuperVersionAndScheduleWork
2639
        // 2) When MutableCFOptions changes. This case is also covered by
I
Igor Canadi 已提交
2640 2641
        // InstallSuperVersionAndScheduleWork, because this is when the new
        // options take effect.
2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652
        // 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 已提交
2653
        }
I
Igor Canadi 已提交
2654 2655
      }
    }
J
jorlow@chromium.org 已提交
2656 2657
  }

2658
  if (!c) {
H
hans@chromium.org 已提交
2659
    // Nothing to do
2660
    LogToBuffer(log_buffer, "Compaction nothing to do");
2661
  } else if (c->deletion_compaction()) {
I
Igor Canadi 已提交
2662 2663 2664 2665
    // 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);
2666
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2667
           kCompactionStyleFIFO);
2668 2669 2670

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

I
Igor Canadi 已提交
2671
    for (const auto& f : *c->inputs(0)) {
2672
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2673
    }
2674 2675 2676
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
2677 2678
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2679 2680 2681
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
2682
    *made_progress = true;
2683
  } else if (!trivial_move_disallowed && c->IsTrivialMove()) {
2684
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2685 2686 2687 2688 2689
    // Instrument for event update
    // TODO(yhchiang): add op details for showing trivial-move.
    ThreadStatusUtil::SetColumnFamily(c->column_family_data());
    ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);

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

2692 2693 2694
    // Move files to next level
    int32_t moved_files = 0;
    int64_t moved_bytes = 0;
2695
    for (unsigned int l = 0; l < c->num_input_levels(); l++) {
2696
      if (c->level(l) == c->output_level()) {
2697 2698 2699 2700
        continue;
      }
      for (size_t i = 0; i < c->num_input_files(l); i++) {
        FileMetaData* f = c->input(l, i);
2701
        c->edit()->DeleteFile(c->level(l), f->fd.GetNumber());
2702 2703 2704 2705 2706 2707 2708 2709 2710 2711 2712 2713
        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();
      }
2714
    }
2715

2716
    status = versions_->LogAndApply(c->column_family_data(),
2717 2718
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2719
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2720 2721
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
2722

S
sdong 已提交
2723
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Islam AbdelRahman 已提交
2724
    c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(),
2725
                                                             moved_bytes);
2726 2727 2728 2729
    {
      event_logger_.LogToBuffer(log_buffer)
          << "job" << job_context->job_id << "event"
          << "trivial_move"
I
Islam AbdelRahman 已提交
2730
          << "destination_level" << c->output_level() << "files" << moved_files
2731
          << "total_files_size" << moved_bytes;
2732
    }
2733 2734
    LogToBuffer(
        log_buffer,
2735
        "[%s] Moved #%d files to level-%d %" PRIu64 " bytes %s: %s\n",
I
Islam AbdelRahman 已提交
2736 2737
        c->column_family_data()->GetName().c_str(), moved_files,
        c->output_level(), moved_bytes, status.ToString().c_str(),
2738
        c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
2739
    *made_progress = true;
2740 2741 2742

    // Clear Instrument
    ThreadStatusUtil::ResetThreadStatus();
J
jorlow@chromium.org 已提交
2743
  } else {
2744 2745 2746
    int output_level  __attribute__((unused)) = c->output_level();
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial",
                             &output_level);
I
Igor Canadi 已提交
2747
    assert(is_snapshot_supported_ || snapshots_.empty());
2748
    CompactionJob compaction_job(
I
Igor Canadi 已提交
2749 2750
        job_context->job_id, c.get(), db_options_, env_options_,
        versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(),
2751
        directories_.GetDataDir(c->output_path_id()), stats_,
2752
        snapshots_.GetAll(), table_cache_, &event_logger_,
2753 2754
        c->mutable_cf_options()->paranoid_file_checks,
        c->mutable_cf_options()->compaction_measure_io_stats, dbname_,
S
sdong 已提交
2755
        &compaction_job_stats);
I
Igor Canadi 已提交
2756
    compaction_job.Prepare();
2757

I
Igor Canadi 已提交
2758
    mutex_.Unlock();
2759
    compaction_job.Run();
2760
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun");
I
Igor Canadi 已提交
2761
    mutex_.Lock();
2762

2763
    status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
I
Igor Canadi 已提交
2764
    if (status.ok()) {
I
Igor Canadi 已提交
2765 2766
      InstallSuperVersionAndScheduleWorkWrapper(
          c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2767
    }
2768
    *made_progress = true;
O
Ori Bernstein 已提交
2769 2770
  }
  if (c != nullptr) {
2771 2772 2773
    NotifyOnCompactionCompleted(
        c->column_family_data(), c.get(), status,
        compaction_job_stats, job_context->job_id);
I
Igor Canadi 已提交
2774
    c->ReleaseCompactionFiles(status);
2775
    *made_progress = true;
J
jorlow@chromium.org 已提交
2776
  }
2777
  // this will unref its input_version and column_family_data
2778
  c.reset();
J
jorlow@chromium.org 已提交
2779 2780 2781

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2782
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2783 2784
    // Ignore compaction errors found during shutting down
  } else {
2785
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2786
        status.ToString().c_str());
2787
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2788 2789 2790
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2791 2792

  if (is_manual) {
G
Gabor Cselle 已提交
2793
    ManualCompaction* m = manual_compaction_;
2794
    if (!status.ok()) {
L
Lei Jin 已提交
2795
      m->status = status;
2796 2797
      m->done = true;
    }
2798 2799 2800 2801 2802 2803 2804 2805 2806
    // 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.
2807 2808 2809 2810 2811
    //
    // 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) {
2812 2813
      m->done = true;
    }
G
Gabor Cselle 已提交
2814 2815 2816
    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 已提交
2817
      // Universal and FIFO compactions should always compact the whole range
S
sdong 已提交
2818 2819 2820
      assert(m->cfd->ioptions()->compaction_style !=
                 kCompactionStyleUniversal ||
             m->cfd->ioptions()->num_levels > 1);
2821
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2822
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2823 2824
      m->begin = &m->tmp_storage;
    }
2825
    m->in_progress = false; // not being processed anymore
2826
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2827
  }
2828
  return status;
J
jorlow@chromium.org 已提交
2829 2830
}

2831 2832
namespace {
struct IterState {
2833
  IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version)
I
Igor Canadi 已提交
2834
      : db(_db), mu(_mu), super_version(_super_version) {}
2835 2836

  DBImpl* db;
2837
  InstrumentedMutex* mu;
2838
  SuperVersion* super_version;
2839 2840 2841 2842
};

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

2844
  if (state->super_version->Unref()) {
2845 2846 2847
    // Job id == 0 means that this is not our background process, but rather
    // user thread
    JobContext job_context(0);
2848

2849 2850
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2851
    state->db->FindObsoleteFiles(&job_context, false, true);
2852 2853 2854
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2855 2856
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2857
    }
I
Igor Canadi 已提交
2858
    job_context.Clean();
I
Igor Canadi 已提交
2859
  }
T
Tomislav Novak 已提交
2860

2861 2862
  delete state;
}
H
Hans Wennborg 已提交
2863
}  // namespace
2864

S
sdong 已提交
2865 2866 2867 2868 2869
InternalIterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
                                              ColumnFamilyData* cfd,
                                              SuperVersion* super_version,
                                              Arena* arena) {
  InternalIterator* internal_iter;
2870 2871 2872 2873 2874
  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 已提交
2875
      super_version->mem->NewIterator(read_options, arena));
2876
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
2877
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
2878
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
2879
  super_version->current->AddIterators(read_options, env_options_,
2880 2881
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
2882
  IterState* cleanup = new IterState(this, &mutex_, super_version);
2883
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2884 2885 2886 2887

  return internal_iter;
}

2888 2889 2890 2891
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2892
Status DBImpl::Get(const ReadOptions& read_options,
2893
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2894
                   std::string* value) {
L
Lei Jin 已提交
2895
  return GetImpl(read_options, column_family, key, value);
2896 2897
}

I
Igor Canadi 已提交
2898 2899
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
2900 2901
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
2902
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
2903
//
I
Igor Canadi 已提交
2904 2905 2906
// However, if InstallSuperVersionAndScheduleWork() gets called twice with the
// same job_context, we can't reuse the SuperVersion() that got
// malloced because
I
Igor Canadi 已提交
2907 2908 2909
// 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 已提交
2910
void DBImpl::InstallSuperVersionAndScheduleWorkWrapper(
I
Igor Canadi 已提交
2911
    ColumnFamilyData* cfd, JobContext* job_context,
2912
    const MutableCFOptions& mutable_cf_options) {
2913
  mutex_.AssertHeld();
I
Igor Canadi 已提交
2914
  SuperVersion* old_superversion = InstallSuperVersionAndScheduleWork(
I
Igor Canadi 已提交
2915 2916 2917
      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 已提交
2918 2919
}

I
Igor Canadi 已提交
2920
SuperVersion* DBImpl::InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
2921
    ColumnFamilyData* cfd, SuperVersion* new_sv,
I
Igor Canadi 已提交
2922
    const MutableCFOptions& mutable_cf_options) {
L
Lei Jin 已提交
2923
  mutex_.AssertHeld();
2924 2925 2926 2927 2928 2929 2930 2931 2932

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

2936
  // Whenever we install new SuperVersion, we might need to issue new flushes or
I
Igor Canadi 已提交
2937 2938 2939 2940
  // compactions.
  SchedulePendingFlush(cfd);
  SchedulePendingCompaction(cfd);
  MaybeScheduleFlushOrCompaction();
L
Lei Jin 已提交
2941 2942 2943 2944 2945 2946 2947 2948 2949 2950

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

2956 2957 2958
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2959
  SequenceNumber snapshot;
L
Lei Jin 已提交
2960 2961 2962
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2963 2964
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2965
  }
2966
  // Acquire SuperVersion
2967
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
2968
  // Prepare to store a list of merge operations if merge occurs.
2969
  MergeContext merge_context;
2970

2971
  Status s;
2972
  // First look in the memtable, then in the immutable memtable (if any).
2973
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2974
  // merge_operands will contain the sequence of merges in the latter case.
2975
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2976
  PERF_TIMER_STOP(get_snapshot_time);
2977

2978
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2979
    // Done
L
Lei Jin 已提交
2980
    RecordTick(stats_, MEMTABLE_HIT);
2981
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2982
    // Done
L
Lei Jin 已提交
2983
    RecordTick(stats_, MEMTABLE_HIT);
2984
  } else {
2985
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2986 2987
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2988
    RecordTick(stats_, MEMTABLE_MISS);
2989
  }
2990

2991 2992
  {
    PERF_TIMER_GUARD(get_post_process_time);
2993

2994
    ReturnAndCleanupSuperVersion(cfd, sv);
2995

2996 2997 2998
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2999
  return s;
J
jorlow@chromium.org 已提交
3000 3001
}

3002
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
3003
    const ReadOptions& read_options,
3004
    const std::vector<ColumnFamilyHandle*>& column_family,
3005
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
3006

L
Lei Jin 已提交
3007
  StopWatch sw(env_, stats_, DB_MULTIGET);
3008
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
3009

3010
  SequenceNumber snapshot;
3011

3012
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
3013
    ColumnFamilyData* cfd;
3014 3015 3016 3017 3018
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
3019 3020 3021 3022 3023 3024
    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});
3025 3026 3027
    }
  }

3028
  mutex_.Lock();
L
Lei Jin 已提交
3029 3030 3031
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
3032 3033 3034
  } else {
    snapshot = versions_->LastSequence();
  }
3035
  for (auto mgd_iter : multiget_cf_data) {
3036 3037
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
3038
  }
3039
  mutex_.Unlock();
3040

3041 3042
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
3043

3044
  // Note: this always resizes the values array
3045 3046 3047
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
3048 3049

  // Keep track of bytes that we read for statistics-recording later
3050
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
3051
  PERF_TIMER_STOP(get_snapshot_time);
3052 3053 3054 3055

  // 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.
3056
  // merge_operands will contain the sequence of merges in the latter case.
3057
  for (size_t i = 0; i < num_keys; ++i) {
3058
    merge_context.Clear();
3059
    Status& s = stat_list[i];
3060 3061 3062
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
3063 3064
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
3065 3066 3067
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
3068
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
3069
      // Done
3070
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
3071 3072
      // Done
    } else {
3073
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
3074 3075
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
3076 3077 3078
    }

    if (s.ok()) {
3079
      bytes_read += value->size();
3080 3081 3082 3083
    }
  }

  // Post processing (decrement reference counts and record statistics)
3084
  PERF_TIMER_GUARD(get_post_process_time);
3085 3086
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
3087
  // TODO(icanadi) do we need lock here or just around Cleanup()?
3088 3089 3090 3091 3092 3093
  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);
3094 3095
    }
  }
3096 3097 3098 3099 3100 3101 3102
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3103
  }
3104

L
Lei Jin 已提交
3105 3106 3107
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3108
  PERF_TIMER_STOP(get_post_process_time);
3109

3110
  return stat_list;
3111 3112
}

3113 3114 3115 3116 3117 3118 3119 3120 3121 3122 3123 3124 3125 3126 3127 3128 3129 3130 3131 3132 3133 3134 3135 3136 3137 3138 3139 3140 3141 3142 3143 3144 3145 3146 3147 3148 3149 3150 3151 3152 3153 3154 3155 3156 3157 3158 3159 3160 3161 3162 3163 3164 3165 3166 3167
#ifndef ROCKSDB_LITE
Status DBImpl::AddFile(ColumnFamilyHandle* column_family,
                       const std::string& file_path, bool move_file) {
  Status status;
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  ColumnFamilyData* cfd = cfh->cfd();

  ExternalSstFileInfo file_info;
  file_info.file_path = file_path;
  status = env_->GetFileSize(file_path, &file_info.file_size);
  if (!status.ok()) {
    return status;
  }

  // Access the file using TableReader to extract
  // version, number of entries, smallest user key, largest user key
  std::unique_ptr<RandomAccessFile> sst_file;
  status = env_->NewRandomAccessFile(file_path, &sst_file, env_options_);
  if (!status.ok()) {
    return status;
  }
  std::unique_ptr<RandomAccessFileReader> sst_file_reader;
  sst_file_reader.reset(new RandomAccessFileReader(std::move(sst_file)));

  std::unique_ptr<TableReader> table_reader;
  status = cfd->ioptions()->table_factory->NewTableReader(
      TableReaderOptions(*cfd->ioptions(), env_options_,
                         cfd->internal_comparator()),
      std::move(sst_file_reader), file_info.file_size, &table_reader);
  if (!status.ok()) {
    return status;
  }

  // Get the external sst file version from table properties
  const UserCollectedProperties& user_collected_properties =
      table_reader->GetTableProperties()->user_collected_properties;
  UserCollectedProperties::const_iterator external_sst_file_version_iter =
      user_collected_properties.find(ExternalSstFilePropertyNames::kVersion);
  if (external_sst_file_version_iter == user_collected_properties.end()) {
    return Status::InvalidArgument("Generated table version not found");
  }

  file_info.version =
      DecodeFixed32(external_sst_file_version_iter->second.c_str());
  if (file_info.version == 1) {
    // version 1 imply that all sequence numbers in table equal 0
    file_info.sequence_number = 0;
  } else {
    return Status::InvalidArgument("Generated table version is not supported");
  }

  // Get number of entries in table
  file_info.num_entries = table_reader->GetTableProperties()->num_entries;

  ParsedInternalKey key;
S
sdong 已提交
3168 3169
  std::unique_ptr<InternalIterator> iter(
      table_reader->NewIterator(ReadOptions()));
3170 3171 3172 3173 3174 3175 3176 3177 3178 3179 3180 3181 3182 3183 3184 3185 3186 3187 3188 3189 3190 3191 3192 3193 3194 3195 3196 3197 3198 3199 3200 3201 3202 3203 3204 3205 3206 3207 3208 3209 3210 3211 3212 3213 3214 3215 3216 3217 3218 3219 3220 3221 3222 3223 3224 3225 3226 3227 3228 3229 3230 3231 3232 3233 3234 3235 3236 3237 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 3282 3283 3284 3285 3286 3287 3288 3289 3290 3291 3292 3293 3294 3295 3296 3297 3298 3299 3300 3301 3302 3303 3304 3305 3306 3307 3308 3309 3310 3311 3312 3313 3314 3315 3316 3317 3318 3319 3320 3321 3322 3323 3324 3325 3326 3327 3328

  // Get first (smallest) key from file
  iter->SeekToFirst();
  if (!ParseInternalKey(iter->key(), &key)) {
    return Status::Corruption("Generated table have corrupted keys");
  }
  if (key.sequence != 0) {
    return Status::Corruption("Generated table have non zero sequence number");
  }
  file_info.smallest_key = key.user_key.ToString();

  // Get last (largest) key from file
  iter->SeekToLast();
  if (!ParseInternalKey(iter->key(), &key)) {
    return Status::Corruption("Generated table have corrupted keys");
  }
  if (key.sequence != 0) {
    return Status::Corruption("Generated table have non zero sequence number");
  }
  file_info.largest_key = key.user_key.ToString();

  return AddFile(column_family, &file_info, move_file);
}

Status DBImpl::AddFile(ColumnFamilyHandle* column_family,
                       const ExternalSstFileInfo* file_info, bool move_file) {
  Status status;
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  ColumnFamilyData* cfd = cfh->cfd();

  if (cfd->NumberLevels() <= 1) {
    return Status::NotSupported(
        "AddFile requires a database with at least 2 levels");
  }
  if (file_info->version != 1) {
    return Status::InvalidArgument("Generated table version is not supported");
  }
  // version 1 imply that file have only Put Operations with Sequence Number = 0

  FileMetaData meta;
  meta.smallest =
      InternalKey(file_info->smallest_key, file_info->sequence_number,
                  ValueType::kTypeValue);
  meta.largest = InternalKey(file_info->largest_key, file_info->sequence_number,
                             ValueType::kTypeValue);
  if (!meta.smallest.Valid() || !meta.largest.Valid()) {
    return Status::Corruption("Generated table have corrupted keys");
  }
  meta.smallest_seqno = file_info->sequence_number;
  meta.largest_seqno = file_info->sequence_number;
  if (meta.smallest_seqno != 0 || meta.largest_seqno != 0) {
    return Status::InvalidArgument(
        "Non zero sequence numbers are not supported");
  }
  // Generate a location for the new table
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, file_info->file_size);
  std::string db_fname = TableFileName(
      db_options_.db_paths, meta.fd.GetNumber(), meta.fd.GetPathId());

  if (move_file) {
    status = env_->LinkFile(file_info->file_path, db_fname);
    if (status.IsNotSupported()) {
      // Original file is on a different FS, use copy instead of hard linking
      status = CopyFile(env_, file_info->file_path, db_fname, 0);
    }
  } else {
    status = CopyFile(env_, file_info->file_path, db_fname, 0);
  }
  if (!status.ok()) {
    return status;
  }

  {
    InstrumentedMutexLock l(&mutex_);
    const MutableCFOptions mutable_cf_options =
        *cfd->GetLatestMutableCFOptions();

    WriteThread::Writer w;
    write_thread_.EnterUnbatched(&w, &mutex_);

    // Make sure memtables are empty
    if (!cfd->mem()->IsEmpty() || cfd->imm()->NumNotFlushed() > 0) {
      // Cannot add the file since the keys in memtable
      // will hide the keys in file
      status = Status::NotSupported("Memtable is not empty");
    }

    // Make sure last sequence number is 0, if there are existing files then
    // they should have sequence number = 0
    if (status.ok() && versions_->LastSequence() > 0) {
      status = Status::NotSupported("Last Sequence number is not zero");
    }

    auto* vstorage = cfd->current()->storage_info();
    if (status.ok()) {
      // Make sure that the key range in the file we will add does not overlap
      // with previously added files
      Slice smallest_user_key = meta.smallest.user_key();
      Slice largest_user_key = meta.largest.user_key();
      for (int level = 0; level < vstorage->num_non_empty_levels(); level++) {
        if (vstorage->OverlapInLevel(level, &smallest_user_key,
                                     &largest_user_key)) {
          status = Status::NotSupported("Cannot add overlapping files");
          break;
        }
      }
    }

    if (status.ok()) {
      // We add the file to the last level
      int target_level = cfd->NumberLevels() - 1;
      if (cfd->ioptions()->level_compaction_dynamic_level_bytes == false) {
        // If we are using dynamic level compaction we add the file to
        // last level with files
        target_level = vstorage->num_non_empty_levels() - 1;
        if (target_level <= 0) {
          target_level = 1;
        }
      }
      VersionEdit edit;
      edit.SetColumnFamily(cfd->GetID());
      edit.AddFile(target_level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                   meta.fd.GetFileSize(), meta.smallest, meta.largest,
                   meta.smallest_seqno, meta.largest_seqno,
                   meta.marked_for_compaction);

      status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                      directories_.GetDbDir());
    }
    write_thread_.ExitUnbatched(&w);

    if (status.ok()) {
      delete InstallSuperVersionAndScheduleWork(cfd, nullptr,
                                                mutable_cf_options);
    }
  }

  if (!status.ok()) {
    // We failed to add the file to the database
    Status s = env_->DeleteFile(db_fname);
    if (!s.ok()) {
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "AddFile() clean up for file %s failed : %s", db_fname.c_str(),
          s.ToString().c_str());
    }
  } else if (status.ok() && move_file) {
    // The file was moved and added successfully, remove original file link
    Status s = env_->DeleteFile(file_info->file_path);
    if (!s.ok()) {
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "%s was added to DB successfully but failed to remove original file "
          "link : %s",
          file_info->file_path.c_str(), s.ToString().c_str());
    }
  }
  return status;
}
#endif  // ROCKSDB_LITE

L
Lei Jin 已提交
3329
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3330
                                  const std::string& column_family_name,
3331
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
3332
  Status s;
I
Igor Canadi 已提交
3333
  *handle = nullptr;
3334 3335 3336 3337 3338 3339

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

Y
Yueh-Hsuan Chiang 已提交
3340
  {
3341
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
3342

Y
Yueh-Hsuan Chiang 已提交
3343 3344 3345 3346 3347 3348 3349 3350 3351 3352 3353 3354 3355 3356
    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 已提交
3357
    {  // write thread
3358 3359
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
I
Igor Canadi 已提交
3360 3361 3362 3363
      // LogAndApply will both write the creation in MANIFEST and create
      // ColumnFamilyData object
      s = versions_->LogAndApply(
          nullptr, MutableCFOptions(opt, ImmutableCFOptions(opt)), &edit,
3364
          &mutex_, directories_.GetDbDir(), false, &cf_options);
3365
      write_thread_.ExitUnbatched(&w);
I
Igor Canadi 已提交
3366
    }
Y
Yueh-Hsuan Chiang 已提交
3367 3368 3369 3370 3371
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
I
Igor Canadi 已提交
3372
      delete InstallSuperVersionAndScheduleWork(
Y
Yueh-Hsuan Chiang 已提交
3373
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
S
sdong 已提交
3374 3375 3376 3377 3378

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

Y
Yueh-Hsuan Chiang 已提交
3379 3380 3381 3382 3383 3384 3385 3386 3387
      *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());
    }
3388
  }  // InstrumentedMutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
3389 3390

  // this is outside the mutex
3391
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
3392 3393
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
3394
  }
3395
  return s;
3396 3397
}

3398 3399 3400 3401
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3402 3403
    return Status::InvalidArgument("Can't drop default column family");
  }
3404

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

I
Igor Canadi 已提交
3407 3408
  VersionEdit edit;
  edit.DropColumnFamily();
3409 3410
  edit.SetColumnFamily(cfd->GetID());

3411
  Status s;
3412
  {
3413
    InstrumentedMutexLock l(&mutex_);
3414 3415 3416 3417
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
3418
      // we drop column family from a single write thread
3419 3420
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
3421 3422
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
3423
      write_thread_.ExitUnbatched(&w);
3424
    }
S
sdong 已提交
3425 3426 3427 3428 3429 3430

    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()) {
3431
        if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) {
S
sdong 已提交
3432 3433 3434 3435 3436 3437
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
3438
  }
3439

3440
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
3441 3442 3443 3444
    // 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 已提交
3445
    assert(cfd->IsDropped());
L
Lei Jin 已提交
3446 3447 3448
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
3449 3450
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
3451
        cfd->GetID());
3452
  } else {
3453
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
3454
        "Dropping column family with id %u FAILED -- %s\n",
3455 3456 3457
        cfd->GetID(), s.ToString().c_str());
  }

3458
  return s;
3459 3460
}

L
Lei Jin 已提交
3461
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
3462 3463
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3464
  if (value_found != nullptr) {
K
Kai Liu 已提交
3465 3466
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3467
  }
L
Lei Jin 已提交
3468
  ReadOptions roptions = read_options;
3469
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3470
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3471

3472
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
3473 3474 3475
  // 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();
3476 3477
}

3478
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
3479 3480 3481
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3482

V
Venkatesh Radhakrishnan 已提交
3483 3484 3485 3486 3487 3488 3489 3490 3491 3492 3493 3494 3495 3496 3497 3498 3499 3500
  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 已提交
3501 3502 3503 3504
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
3505 3506
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
3507
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3508 3509 3510
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
3511
#endif
T
Tomislav Novak 已提交
3512
  } else {
3513
    SequenceNumber latest_snapshot = versions_->LastSequence();
3514
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
3515

I
Igor Canadi 已提交
3516
    auto snapshot =
3517 3518 3519
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
3520
            : latest_snapshot;
T
Tomislav Novak 已提交
3521

3522 3523 3524 3525 3526 3527 3528 3529 3530 3531 3532 3533 3534 3535 3536 3537 3538 3539 3540 3541 3542 3543 3544 3545 3546 3547 3548 3549 3550 3551 3552 3553 3554 3555 3556 3557
    // 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         | <----+
    // |       |   +-------------------+
    // |       |                       |
    // +-------+-----------------------+
    //
3558 3559
    // ArenaWrappedDBIter inlines an arena area where all the iterators in
    // the iterator tree are allocated in the order of being accessed when
3560 3561 3562 3563 3564
    // 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(
3565
        env_, *cfd->ioptions(), cfd->user_comparator(),
3566
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
3567
        read_options.iterate_upper_bound);
3568

S
sdong 已提交
3569
    InternalIterator* internal_iter =
3570
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3571 3572 3573 3574
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
3575 3576
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
3577 3578
}

3579
Status DBImpl::NewIterators(
3580
    const ReadOptions& read_options,
I
Igor Canadi 已提交
3581
    const std::vector<ColumnFamilyHandle*>& column_families,
3582
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3583 3584
  iterators->clear();
  iterators->reserve(column_families.size());
V
Venkatesh Radhakrishnan 已提交
3585 3586 3587 3588 3589 3590 3591 3592 3593 3594 3595 3596 3597 3598 3599 3600 3601 3602 3603 3604
  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 已提交
3605 3606 3607 3608
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3609 3610
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
3611 3612
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
3613
      iterators->push_back(
3614
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3615 3616
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
3617
    }
I
Igor Canadi 已提交
3618
#endif
I
Igor Canadi 已提交
3619
  } else {
3620 3621
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
3622
    for (size_t i = 0; i < column_families.size(); ++i) {
3623 3624 3625
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
3626 3627

      auto snapshot =
3628 3629 3630
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
3631 3632
              : latest_snapshot;

3633
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3634
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
3635
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
S
sdong 已提交
3636 3637
      InternalIterator* internal_iter =
          NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3638 3639
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3640 3641 3642 3643
    }
  }

  return Status::OK();
3644 3645
}

J
jorlow@chromium.org 已提交
3646
const Snapshot* DBImpl::GetSnapshot() {
3647 3648
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error
3649
  SnapshotImpl* s = new SnapshotImpl;
3650

3651
  InstrumentedMutexLock l(&mutex_);
3652
  // returns null if the underlying memtable does not support snapshot.
3653 3654 3655 3656 3657
  if (!is_snapshot_supported_) {
    delete s;
    return nullptr;
  }
  return snapshots_.New(s, versions_->LastSequence(), unix_time);
J
jorlow@chromium.org 已提交
3658 3659 3660
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
3661 3662 3663 3664 3665 3666
  const SnapshotImpl* casted_s = reinterpret_cast<const SnapshotImpl*>(s);
  {
    InstrumentedMutexLock l(&mutex_);
    snapshots_.Delete(casted_s);
  }
  delete casted_s;
J
jorlow@chromium.org 已提交
3667 3668 3669
}

// Convenience methods
3670 3671
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3672
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3673 3674
}

3675 3676 3677
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
3678
  if (!cfh->cfd()->ioptions()->merge_operator) {
3679 3680
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3681
    return DB::Merge(o, column_family, key, val);
3682 3683 3684
  }
}

L
Lei Jin 已提交
3685
Status DBImpl::Delete(const WriteOptions& write_options,
3686
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
3687
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
3688 3689
}

A
Andres Noetzli 已提交
3690 3691 3692 3693 3694 3695
Status DBImpl::SingleDelete(const WriteOptions& write_options,
                            ColumnFamilyHandle* column_family,
                            const Slice& key) {
  return DB::SingleDelete(write_options, column_family, key);
}

L
Lei Jin 已提交
3696
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
A
agiardullo 已提交
3697 3698 3699
  return WriteImpl(write_options, my_batch, nullptr);
}

A
agiardullo 已提交
3700
#ifndef ROCKSDB_LITE
A
agiardullo 已提交
3701 3702 3703 3704 3705
Status DBImpl::WriteWithCallback(const WriteOptions& write_options,
                                 WriteBatch* my_batch,
                                 WriteCallback* callback) {
  return WriteImpl(write_options, my_batch, callback);
}
A
agiardullo 已提交
3706
#endif  // ROCKSDB_LITE
A
agiardullo 已提交
3707 3708 3709

Status DBImpl::WriteImpl(const WriteOptions& write_options,
                         WriteBatch* my_batch, WriteCallback* callback) {
S
Stanislau Hlebik 已提交
3710 3711 3712
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
3713 3714 3715
  if (write_options.timeout_hint_us != 0) {
    return Status::InvalidArgument("timeout_hint_us is deprecated");
  }
A
agiardullo 已提交
3716 3717

  Status status;
A
agiardullo 已提交
3718 3719
  bool callback_failed = false;

A
agiardullo 已提交
3720 3721 3722 3723 3724 3725 3726 3727 3728 3729
  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;
  }

3730
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
3731
  WriteThread::Writer w;
S
Stanislau Hlebik 已提交
3732
  w.batch = my_batch;
L
Lei Jin 已提交
3733 3734
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
3735 3736
  w.in_batch_group = false;
  w.done = false;
A
agiardullo 已提交
3737
  w.has_callback = (callback != nullptr) ? true : false;
S
Stanislau Hlebik 已提交
3738

L
Lei Jin 已提交
3739
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
3740 3741 3742
    RecordTick(stats_, WRITE_WITH_WAL);
  }

3743 3744
  StopWatch write_sw(env_, db_options_.statistics.get(), DB_WRITE);

3745 3746 3747 3748 3749 3750 3751 3752
  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 已提交
3753 3754
  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
3755 3756 3757 3758 3759

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

S
Stanislau Hlebik 已提交
3760 3761
  RecordTick(stats_, WRITE_DONE_BY_SELF);
  default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_SELF, 1);
3762

3763 3764 3765 3766
  // 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.
3767 3768 3769
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

3770
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
3771
                                    ? 4 * max_total_in_memory_state_
3772
                                    : db_options_.max_total_wal_size;
3773
  if (UNLIKELY(!single_column_family_mode_) &&
3774
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3775
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
3776
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
3777
    alive_log_files_.begin()->getting_flushed = true;
3778
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3779 3780 3781
        "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 已提交
3782 3783
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
3784
    for (auto cfd : *versions_->GetColumnFamilySet()) {
3785 3786 3787
      if (cfd->IsDropped()) {
        continue;
      }
I
Igor Canadi 已提交
3788
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
I
Igor Canadi 已提交
3789
        status = SwitchMemtable(cfd, &context);
I
Igor Canadi 已提交
3790 3791 3792
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
3793
        cfd->imm()->FlushRequested();
3794
        SchedulePendingFlush(cfd);
3795
      }
3796
    }
I
Igor Canadi 已提交
3797
    MaybeScheduleFlushOrCompaction();
3798 3799 3800 3801 3802 3803 3804 3805
  } 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()) {
3806 3807 3808
      if (cfd->IsDropped()) {
        continue;
      }
3809
      if (!cfd->mem()->IsEmpty()) {
I
Igor Canadi 已提交
3810
        status = SwitchMemtable(cfd, &context);
3811 3812 3813 3814
        if (!status.ok()) {
          break;
        }
        cfd->imm()->FlushRequested();
3815
        SchedulePendingFlush(cfd);
3816 3817 3818
      }
    }
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
3819 3820 3821 3822 3823 3824 3825 3826 3827 3828
  }

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

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

S
sdong 已提交
3829 3830
  if (UNLIKELY(status.ok()) &&
      (write_controller_.IsStopped() || write_controller_.NeedsDelay())) {
3831 3832
    PERF_TIMER_STOP(write_pre_and_post_process_time);
    PERF_TIMER_GUARD(write_delay_time);
S
sdong 已提交
3833 3834 3835 3836
    // 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.
3837
    status = DelayWrite(last_batch_group_size_);
3838
    PERF_TIMER_START(write_pre_and_post_process_time);
I
Igor Canadi 已提交
3839 3840
  }

D
dgrogan@chromium.org 已提交
3841
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
3842
  WriteThread::Writer* last_writer = &w;
A
agiardullo 已提交
3843
  autovector<WriteBatch*> write_batch_group;
3844 3845
  bool need_log_sync = !write_options.disableWAL && write_options.sync;
  bool need_log_dir_sync = need_log_sync && !log_dir_synced_;
A
agiardullo 已提交
3846

S
Stanislau Hlebik 已提交
3847
  if (status.ok()) {
3848 3849
    last_batch_group_size_ = write_thread_.EnterAsBatchGroupLeader(
        &w, &last_writer, &write_batch_group);
3850

3851
    if (need_log_sync) {
3852 3853 3854 3855 3856 3857 3858 3859 3860
      while (logs_.front().getting_synced) {
        log_sync_cv_.Wait();
      }
      for (auto& log : logs_) {
        assert(!log.getting_synced);
        log.getting_synced = true;
      }
    }

3861 3862 3863
    // 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
3864
    // into memtables
A
agiardullo 已提交
3865 3866 3867 3868 3869 3870 3871

    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 已提交
3872
      callback_failed = true;
A
agiardullo 已提交
3873 3874 3875 3876 3877 3878 3879 3880
    }
  } else {
    mutex_.Unlock();
  }

  // At this point the mutex is unlocked

  if (status.ok()) {
3881 3882 3883 3884 3885 3886 3887 3888 3889 3890
      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]);
        }
      }

3891 3892 3893 3894
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
3895
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
3896
      // Record statistics
L
Lei Jin 已提交
3897
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
3898
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
3899
      if (write_options.disableWAL) {
3900
        flush_on_destroy_ = true;
3901
      }
L
Lei Jin 已提交
3902
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3903

3904
      uint64_t log_size = 0;
L
Lei Jin 已提交
3905
      if (!write_options.disableWAL) {
3906
        PERF_TIMER_GUARD(write_wal_time);
3907
        Slice log_entry = WriteBatchInternal::Contents(updates);
3908
        status = logs_.back().writer->AddRecord(log_entry);
I
Igor Canadi 已提交
3909 3910
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
3911
        log_empty_ = false;
3912
        log_size = log_entry.size();
L
Lei Jin 已提交
3913
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
3914
        if (status.ok() && need_log_sync) {
I
Igor Canadi 已提交
3915
          RecordTick(stats_, WAL_FILE_SYNCED);
3916
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
3917 3918 3919 3920 3921 3922 3923 3924
          // 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_) {
3925
            status = log.writer->file()->Sync(db_options_.use_fsync);
3926 3927 3928 3929
            if (!status.ok()) {
              break;
            }
          }
3930
          if (status.ok() && need_log_dir_sync) {
3931 3932 3933 3934 3935
            // 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 已提交
3936
        }
3937 3938
      }
      if (status.ok()) {
3939
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
3940

3941
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3942
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
3943
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
3944 3945 3946 3947 3948 3949 3950 3951
        // 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 已提交
3952
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
3953
      }
L
Lei Jin 已提交
3954
      PERF_TIMER_START(write_pre_and_post_process_time);
3955 3956 3957
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
3958
      mutex_.Lock();
A
agiardullo 已提交
3959

3960 3961 3962
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
3963 3964
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
3965
      if (!write_options.disableWAL) {
3966 3967 3968 3969
        if (write_options.sync) {
          default_cf_internal_stats_->AddDBStats(InternalStats::WAL_FILE_SYNCED,
                                                 1);
        }
3970 3971 3972
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
3973
      if (status.ok()) {
3974
        versions_->SetLastSequence(last_sequence);
3975
      }
A
agiardullo 已提交
3976 3977 3978
  } else {
    // Operation failed.  Make sure sure mutex is held for cleanup code below.
    mutex_.Lock();
3979
  }
A
agiardullo 已提交
3980

A
agiardullo 已提交
3981
  if (db_options_.paranoid_checks && !status.ok() && !callback_failed &&
3982
      !status.IsBusy() && bg_error_.ok()) {
I
Igor Canadi 已提交
3983 3984
    bg_error_ = status; // stop compaction & fail any further writes
  }
3985

A
agiardullo 已提交
3986
  mutex_.AssertHeld();
3987

3988 3989
  if (need_log_sync) {
    MarkLogsSynced(logfile_number_, need_log_dir_sync, status);
3990 3991
  }

3992 3993 3994 3995 3996 3997 3998 3999 4000
  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);
    }
  }
4001

I
Igor Canadi 已提交
4002
  mutex_.Unlock();
4003

4004 4005
  write_thread_.ExitAsBatchGroupLeader(&w, last_writer, status);

J
jorlow@chromium.org 已提交
4006 4007 4008
  return status;
}

4009 4010
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
4011
Status DBImpl::DelayWrite(uint64_t num_bytes) {
S
sdong 已提交
4012 4013 4014 4015
  uint64_t time_delayed = 0;
  bool delayed = false;
  {
    StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed);
S
sdong 已提交
4016 4017
    auto delay = write_controller_.GetDelay(env_, num_bytes);
    if (delay > 0) {
S
sdong 已提交
4018 4019
      mutex_.Unlock();
      delayed = true;
S
sdong 已提交
4020
      TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep");
4021 4022
      // hopefully we don't have to sleep more than 2 billion microseconds
      env_->SleepForMicroseconds(static_cast<int>(delay));
S
sdong 已提交
4023 4024
      mutex_.Lock();
    }
4025

4026
    while (bg_error_.ok() && write_controller_.IsStopped()) {
S
sdong 已提交
4027
      delayed = true;
4028 4029
      TEST_SYNC_POINT("DBImpl::DelayWrite:Wait");
      bg_cv_.Wait();
4030 4031
    }
  }
S
sdong 已提交
4032 4033 4034 4035 4036
  if (delayed) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_STALL_MICROS,
                                           time_delayed);
    RecordTick(stats_, STALL_MICROS, time_delayed);
  }
I
Igor Canadi 已提交
4037 4038

  return bg_error_;
4039 4040
}

I
Igor Canadi 已提交
4041 4042 4043
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
I
Igor Canadi 已提交
4044
    auto status = SwitchMemtable(cfd, context);
I
Igor Canadi 已提交
4045 4046 4047 4048 4049
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
4050 4051
    }
  }
I
Igor Canadi 已提交
4052
  return Status::OK();
S
Stanislau Hlebik 已提交
4053 4054 4055 4056
}

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
4057
Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
S
Stanislau Hlebik 已提交
4058 4059 4060 4061 4062 4063 4064
  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.
4065
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
4066 4067 4068 4069
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
4070
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
4071 4072 4073 4074
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
4075 4076
      EnvOptions opt_env_opt =
          env_->OptimizeForLogWrite(env_options_, db_options_);
4077
      s = env_->NewWritableFile(
4078
          LogFileName(db_options_.wal_dir, new_log_number), &lfile,
4079
          opt_env_opt);
S
Stanislau Hlebik 已提交
4080 4081 4082
      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 已提交
4083 4084
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
4085 4086 4087
        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 已提交
4088
      }
S
Stanislau Hlebik 已提交
4089 4090 4091
    }

    if (s.ok()) {
A
agiardullo 已提交
4092 4093
      SequenceNumber seq = versions_->LastSequence();
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options, seq);
S
Stanislau Hlebik 已提交
4094 4095 4096
      new_superversion = new SuperVersion();
    }
  }
4097
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
4098 4099
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
4100 4101 4102 4103 4104 4105 4106 4107 4108 4109 4110 4111
  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;
4112
    log_dir_synced_ = false;
4113
    logs_.emplace_back(logfile_number_, new_log);
S
Stanislau Hlebik 已提交
4114
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
4115
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
4116 4117 4118 4119
      // 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 已提交
4120
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
4121
          loop_cfd->imm()->NumNotFlushed() == 0) {
I
Igor Canadi 已提交
4122
        loop_cfd->SetLogNumber(logfile_number_);
4123
      }
4124 4125
    }
  }
S
Stanislau Hlebik 已提交
4126
  cfd->mem()->SetNextLogNumber(logfile_number_);
4127
  cfd->imm()->Add(cfd->mem(), &context->memtables_to_free_);
S
Stanislau Hlebik 已提交
4128 4129
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
I
Igor Canadi 已提交
4130 4131
  context->superversions_to_free_.push_back(InstallSuperVersionAndScheduleWork(
      cfd, new_superversion, mutable_cf_options));
4132 4133 4134
  return s;
}

I
Igor Canadi 已提交
4135
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
4136 4137 4138 4139 4140
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

4141 4142
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
4143
  auto version = cfd->current();
4144 4145 4146 4147 4148 4149 4150 4151 4152 4153 4154 4155
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
4156
#endif  // ROCKSDB_LITE
4157

I
Igor Canadi 已提交
4158 4159 4160 4161
const std::string& DBImpl::GetName() const {
  return dbname_;
}

4162 4163 4164 4165
Env* DBImpl::GetEnv() const {
  return env_;
}

4166 4167
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
4168
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
4169 4170
}

4171 4172
const DBOptions& DBImpl::GetDBOptions() const { return db_options_; }

4173
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
4174
                         const Slice& property, std::string* value) {
4175 4176
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4177 4178 4179
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

4180
  value->clear();
4181 4182 4183 4184 4185
  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) {
4186
      *value = ToString(int_value);
4187 4188 4189 4190 4191
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
4192
    InstrumentedMutexLock l(&mutex_);
4193 4194 4195
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
4196 4197
}

4198 4199
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
4200 4201
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4202 4203 4204 4205 4206 4207 4208 4209 4210 4211 4212 4213
  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) {
4214 4215
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4216 4217

  if (!need_out_of_mutex) {
4218
    InstrumentedMutexLock l(&mutex_);
4219
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
4220 4221 4222 4223 4224 4225 4226 4227 4228 4229 4230 4231 4232 4233
  } 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 已提交
4234
  return cfd->GetThreadLocalSuperVersion(&mutex_);
4235 4236
}

A
agiardullo 已提交
4237 4238 4239 4240 4241 4242 4243 4244 4245 4246 4247 4248
// 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 已提交
4249 4250 4251 4252 4253 4254 4255 4256 4257 4258 4259 4260 4261 4262 4263 4264
// 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);
}

4265 4266
void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
4267
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
4268 4269 4270 4271 4272

  if (unref_sv) {
    // Release SuperVersion
    if (sv->Unref()) {
      {
4273
        InstrumentedMutexLock l(&mutex_);
4274 4275 4276 4277 4278 4279 4280
        sv->Cleanup();
      }
      delete sv;
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
    }
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
  }
4281 4282
}

A
agiardullo 已提交
4283 4284 4285 4286 4287 4288 4289 4290 4291 4292 4293 4294
// 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 已提交
4295 4296 4297 4298 4299 4300 4301 4302 4303 4304 4305 4306 4307 4308 4309 4310
// 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 已提交
4311 4312 4313 4314 4315 4316 4317 4318 4319 4320 4321 4322
// 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 已提交
4323 4324 4325 4326 4327 4328 4329 4330 4331 4332 4333 4334 4335 4336
// 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();
}

4337
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4338 4339
                                 const Range* range, int n, uint64_t* sizes,
                                 bool include_memtable) {
J
jorlow@chromium.org 已提交
4340
  Version* v;
4341 4342
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4343 4344
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
  v = sv->current;
J
jorlow@chromium.org 已提交
4345 4346 4347

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
4348 4349 4350
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
    sizes[i] = versions_->ApproximateSize(v, k1.Encode(), k2.Encode());
4351 4352 4353 4354
    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 已提交
4355 4356
  }

4357
  ReturnAndCleanupSuperVersion(cfd, sv);
J
jorlow@chromium.org 已提交
4358 4359
}

I
Igor Canadi 已提交
4360 4361 4362 4363 4364 4365 4366 4367 4368 4369 4370 4371 4372 4373 4374 4375
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 已提交
4376 4377 4378 4379 4380
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
4381
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
4382 4383 4384
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
4385
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
4386 4387
}

4388 4389 4390
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4391 4392 4393
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
4394 4395
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
4396 4397 4398
    return Status::InvalidArgument("Invalid file name");
  }

4399 4400 4401 4402
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4403 4404
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
4405
          name.c_str());
4406 4407
      return Status::NotSupported("Delete only supported for archived logs");
    }
4408
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
4409
    if (!status.ok()) {
4410 4411
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
4412
          name.c_str(), status.ToString().c_str());
4413 4414 4415 4416
    }
    return status;
  }

4417
  int level;
I
Igor Canadi 已提交
4418
  FileMetaData* metadata;
4419
  ColumnFamilyData* cfd;
4420
  VersionEdit edit;
4421
  JobContext job_context(next_job_id_.fetch_add(1), true);
D
Dhruba Borthakur 已提交
4422
  {
4423
    InstrumentedMutexLock l(&mutex_);
4424
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4425
    if (!status.ok()) {
4426 4427
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
4428
      job_context.Clean();
D
Dhruba Borthakur 已提交
4429 4430
      return Status::InvalidArgument("File not found");
    }
4431
    assert(level < cfd->NumberLevels());
4432

D
Dhruba Borthakur 已提交
4433
    // If the file is being compacted no need to delete.
4434
    if (metadata->being_compacted) {
4435
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
4436
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
4437
      job_context.Clean();
D
Dhruba Borthakur 已提交
4438
      return Status::OK();
4439 4440
    }

D
Dhruba Borthakur 已提交
4441 4442 4443
    // 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 已提交
4444
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
4445
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
4446
      if (vstoreage->NumLevelFiles(i) != 0) {
4447
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
4448
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
4449
        job_context.Clean();
D
Dhruba Borthakur 已提交
4450 4451 4452
        return Status::InvalidArgument("File not in last level");
      }
    }
4453
    // if level == 0, it has to be the oldest file
S
sdong 已提交
4454 4455
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
4456 4457
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
4458
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
4459
      job_context.Clean();
4460 4461 4462
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
4463
    edit.DeleteFile(level, number);
4464
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
4465
                                    &edit, &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
4466
    if (status.ok()) {
I
Igor Canadi 已提交
4467 4468
      InstallSuperVersionAndScheduleWorkWrapper(
          cfd, &job_context, *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
4469
    }
I
Igor Canadi 已提交
4470 4471
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
4472

4473
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
4474
  // remove files outside the db-lock
I
Igor Canadi 已提交
4475
  if (job_context.HaveSomethingToDelete()) {
4476
    // Call PurgeObsoleteFiles() without holding mutex.
I
Igor Canadi 已提交
4477
    PurgeObsoleteFiles(job_context);
4478
  }
I
Igor Canadi 已提交
4479
  job_context.Clean();
4480 4481 4482
  return status;
}

4483
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
4484
  InstrumentedMutexLock l(&mutex_);
4485
  versions_->GetLiveFilesMetaData(metadata);
4486
}
4487 4488 4489 4490 4491 4492 4493 4494 4495 4496 4497

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 已提交
4498
#endif  // ROCKSDB_LITE
4499

I
Igor Canadi 已提交
4500 4501 4502 4503 4504 4505 4506
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
4510 4511
    uint64_t fsize = 0;
    Status s = env_->GetFileSize(file_path, &fsize);
D
dyniusz 已提交
4512 4513 4514 4515
    if (!s.ok() &&
        env_->GetFileSize(Rocks2LevelTableFileName(file_path), &fsize).ok()) {
      s = Status::OK();
    }
I
Igor Canadi 已提交
4516 4517 4518 4519
    if (!s.ok()) {
      corruption_messages +=
          "Can't access " + md.name + ": " + s.ToString() + "\n";
    } else if (fsize != md.size) {
4520
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
4521
                             ". Size recorded in manifest " +
4522 4523
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
4524 4525 4526 4527 4528 4529 4530 4531 4532
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

4533
Status DBImpl::GetDbIdentity(std::string& identity) const {
4534 4535
  std::string idfilename = IdentityFileName(dbname_);
  const EnvOptions soptions;
4536 4537 4538 4539 4540 4541 4542 4543 4544
  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)));
4545
  }
4546

4547 4548 4549 4550 4551
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
D
Dmitri Smirnov 已提交
4552
  char* buffer = reinterpret_cast<char*>(alloca(file_size));
4553
  Slice id;
4554
  s = id_file_reader->Read(static_cast<size_t>(file_size), &id, buffer);
4555 4556 4557 4558 4559 4560 4561 4562 4563 4564 4565
  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 已提交
4566 4567
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
4568
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
4569
               const Slice& key, const Slice& value) {
4570 4571 4572 4573
  // 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);
4574
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
4575 4576 4577
  return Write(opt, &batch);
}

4578 4579
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
4580
  WriteBatch batch;
4581
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
4582 4583 4584
  return Write(opt, &batch);
}

A
Andres Noetzli 已提交
4585 4586 4587 4588 4589 4590 4591
Status DB::SingleDelete(const WriteOptions& opt,
                        ColumnFamilyHandle* column_family, const Slice& key) {
  WriteBatch batch;
  batch.SingleDelete(column_family, key);
  return Write(opt, &batch);
}

4592 4593
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
4594
  WriteBatch batch;
4595
  batch.Merge(column_family, key, value);
4596 4597 4598
  return Write(opt, &batch);
}

4599
// Default implementation -- returns not supported status
L
Lei Jin 已提交
4600
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
4601
                              const std::string& column_family_name,
4602
                              ColumnFamilyHandle** handle) {
4603
  return Status::NotSupported("");
4604
}
4605
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
4606
  return Status::NotSupported("");
4607 4608
}

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

J
Jim Paton 已提交
4611
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
4612 4613 4614 4615
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
4616
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
4617
  std::vector<ColumnFamilyHandle*> handles;
4618
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
4619 4620 4621 4622 4623 4624 4625
  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;
4626 4627
}

4628 4629
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
4630
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
4631
  Status s = SanitizeOptionsByTable(db_options, column_families);
4632 4633 4634
  if (!s.ok()) {
    return s;
  }
4635

4636 4637 4638 4639 4640 4641
  for (auto& cfd : column_families) {
    s = CheckCompressionSupported(cfd.options);
    if (!s.ok()) {
      return s;
    }
    if (db_options.db_paths.size() > 1) {
4642 4643
      if ((cfd.options.compaction_style != kCompactionStyleUniversal) &&
          (cfd.options.compaction_style != kCompactionStyleLevel)) {
4644 4645
        return Status::NotSupported(
            "More than one DB paths are only supported in "
4646
            "universal and level compaction styles. ");
4647 4648
      }
    }
4649
  }
4650

4651 4652 4653
  if (db_options.db_paths.size() > 4) {
    return Status::NotSupported(
        "More than four DB paths are not supported yet. ");
4654 4655
  }

4656
  *dbptr = nullptr;
4657
  handles->clear();
J
jorlow@chromium.org 已提交
4658

I
Igor Canadi 已提交
4659 4660 4661 4662
  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);
4663
  }
4664

I
Igor Canadi 已提交
4665
  DBImpl* impl = new DBImpl(db_options, dbname);
4666
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
4667
  if (s.ok()) {
4668
    for (auto db_path : impl->db_options_.db_paths) {
4669
      s = impl->env_->CreateDirIfMissing(db_path.path);
4670 4671 4672 4673 4674 4675
      if (!s.ok()) {
        break;
      }
    }
  }

4676 4677 4678 4679 4680 4681
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
4682 4683 4684 4685
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
4686
  impl->mutex_.Lock();
4687 4688
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
4689
  if (s.ok()) {
4690
    uint64_t new_log_number = impl->versions_->NewFileNumber();
4691
    unique_ptr<WritableFile> lfile;
4692
    EnvOptions soptions(db_options);
4693 4694
    EnvOptions opt_env_options =
        impl->db_options_.env->OptimizeForLogWrite(soptions, impl->db_options_);
4695 4696
    s = impl->db_options_.env->NewWritableFile(
        LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
4697
        opt_env_options);
J
jorlow@chromium.org 已提交
4698
    if (s.ok()) {
4699
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
4700
      impl->logfile_number_ = new_log_number;
4701 4702
      unique_ptr<WritableFileWriter> file_writer(
          new WritableFileWriter(std::move(lfile), opt_env_options));
4703 4704
      impl->logs_.emplace_back(new_log_number,
                               new log::Writer(std::move(file_writer)));
I
Igor Canadi 已提交
4705

4706 4707
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
4708 4709
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
4710 4711 4712
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
4713
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
4714 4715 4716 4717 4718 4719 4720 4721 4722 4723 4724 4725 4726 4727 4728 4729
        } 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 已提交
4730
        }
4731
      }
I
Igor Canadi 已提交
4732 4733
    }
    if (s.ok()) {
4734
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
4735
        delete impl->InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
4736
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
4737
      }
I
Igor Canadi 已提交
4738 4739
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4740
      impl->DeleteObsoleteFiles();
4741
      s = impl->directories_.GetDbDir()->Fsync();
J
jorlow@chromium.org 已提交
4742 4743
    }
  }
4744

I
Igor Canadi 已提交
4745 4746
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
S
sdong 已提交
4747
      if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
4748
        auto* vstorage = cfd->current()->storage_info();
4749
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
4750
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
4751
          if (num_files > 0) {
I
Igor Canadi 已提交
4752 4753
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
S
sdong 已提交
4754
                "open with FIFO compaction style.");
I
Igor Canadi 已提交
4755 4756 4757 4758
            break;
          }
        }
      }
S
sdong 已提交
4759 4760 4761
      if (!cfd->mem()->IsSnapshotSupported()) {
        impl->is_snapshot_supported_ = false;
      }
4762
      if (cfd->ioptions()->merge_operator != nullptr &&
4763 4764 4765 4766 4767
          !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 已提交
4768
      if (!s.ok()) {
4769 4770 4771 4772
        break;
      }
    }
  }
4773 4774 4775 4776 4777
  TEST_SYNC_POINT("DBImpl::Open:Opened");
  if (s.ok()) {
    impl->opened_successfully_ = true;
    impl->MaybeScheduleFlushOrCompaction();
  }
4778 4779
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4780
  if (s.ok()) {
S
sdong 已提交
4781 4782
    Log(InfoLogLevel::INFO_LEVEL, impl->db_options_.info_log, "DB pointer %p",
        impl);
4783 4784
    LogFlush(impl->db_options_.info_log);

J
jorlow@chromium.org 已提交
4785 4786
    *dbptr = impl;
  } else {
Y
Yueh-Hsuan Chiang 已提交
4787
    for (auto* h : *handles) {
4788 4789
      delete h;
    }
4790
    handles->clear();
J
jorlow@chromium.org 已提交
4791 4792 4793 4794 4795
    delete impl;
  }
  return s;
}

4796 4797 4798
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4799
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4800 4801
}

4802 4803 4804
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4805
Status DestroyDB(const std::string& dbname, const Options& options) {
4806
  const InternalKeyComparator comparator(options.comparator);
4807
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
4808
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4809
  std::vector<std::string> filenames;
4810

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

J
jorlow@chromium.org 已提交
4814
  FileLock* lock;
4815 4816
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4817 4818 4819
  if (result.ok()) {
    uint64_t number;
    FileType type;
4820
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
4821
    for (size_t i = 0; i < filenames.size(); i++) {
4822
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
4823
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4824
        Status del;
4825
        std::string path_to_delete = dbname + "/" + filenames[i];
K
Kosie van der Merwe 已提交
4826
        if (type == kMetaDatabase) {
4827 4828 4829
          del = DestroyDB(path_to_delete, options);
        } else if (type == kTableFile) {
          del = DeleteOrMoveToTrash(&options, path_to_delete);
K
Kosie van der Merwe 已提交
4830
        } else {
4831
          del = env->DeleteFile(path_to_delete);
K
Kosie van der Merwe 已提交
4832
        }
J
jorlow@chromium.org 已提交
4833 4834 4835 4836 4837
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4838

4839 4840
    for (size_t path_id = 0; path_id < options.db_paths.size(); path_id++) {
      const auto& db_path = options.db_paths[path_id];
4841
      env->GetChildren(db_path.path, &filenames);
4842 4843 4844
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
4845 4846 4847 4848 4849 4850 4851
          Status del;
          std::string table_path = db_path.path + "/" + filenames[i];
          if (path_id == 0) {
            del = DeleteOrMoveToTrash(&options, table_path);
          } else {
            del = env->DeleteFile(table_path);
          }
4852 4853 4854 4855 4856 4857 4858
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

I
Igor Canadi 已提交
4859 4860 4861 4862 4863 4864 4865 4866 4867 4868 4869 4870 4871 4872 4873 4874 4875 4876
    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;
4877
    env->GetChildren(archivedir, &archiveFiles);
4878 4879
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4880 4881
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4882
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4883 4884 4885 4886 4887
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4888
    // ignore case where no archival directory is present.
4889
    env->DeleteDir(archivedir);
4890

J
jorlow@chromium.org 已提交
4891
    env->UnlockFile(lock);  // Ignore error since state is already gone
4892
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4893
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4894
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4895 4896 4897 4898
  }
  return result;
}

Y
Yueh-Hsuan Chiang 已提交
4899
#if ROCKSDB_USING_THREAD_STATUS
4900

Y
Yueh-Hsuan Chiang 已提交
4901 4902
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4903
  if (db_options_.enable_thread_tracking) {
4904
    ThreadStatusUtil::NewColumnFamilyInfo(this, cfd);
4905
  }
Y
Yueh-Hsuan Chiang 已提交
4906 4907 4908 4909
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4910
  if (db_options_.enable_thread_tracking) {
4911
    ThreadStatusUtil::EraseColumnFamilyInfo(cfd);
4912
  }
Y
Yueh-Hsuan Chiang 已提交
4913 4914 4915
}

void DBImpl::EraseThreadStatusDbInfo() const {
4916
  if (db_options_.enable_thread_tracking) {
4917
    ThreadStatusUtil::EraseDatabaseInfo(this);
4918
  }
Y
Yueh-Hsuan Chiang 已提交
4919 4920 4921 4922 4923 4924 4925 4926 4927 4928 4929 4930 4931 4932 4933
}

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

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

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

4934 4935
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
4936
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4937
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
4938 4939
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
4940 4941 4942 4943
  Header(log, "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
         ROCKSDB_PATCH);
  Header(log, "Git sha %s", rocksdb_build_git_sha);
  Header(log, "Compile date %s", rocksdb_build_compile_date);
I
Igor Canadi 已提交
4944
#endif
4945 4946
}

A
agiardullo 已提交
4947 4948 4949 4950 4951 4952 4953 4954 4955 4956 4957 4958 4959 4960 4961 4962 4963 4964 4965 4966 4967 4968 4969 4970 4971 4972 4973 4974 4975 4976 4977 4978 4979 4980 4981 4982 4983 4984 4985 4986 4987 4988 4989 4990 4991 4992 4993 4994 4995 4996 4997 4998 4999 5000 5001 5002 5003 5004 5005 5006 5007 5008 5009 5010 5011 5012 5013 5014 5015 5016 5017 5018 5019 5020 5021 5022 5023 5024 5025
#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

5026
}  // namespace rocksdb