db_impl.cc 199.9 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
#include <stdint.h>
D
David Bernard 已提交
18
#ifdef OS_SOLARIS
D
David Bernard 已提交
19
#include <alloca.h>
D
David Bernard 已提交
20
#endif
21

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

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

102
namespace rocksdb {
J
jorlow@chromium.org 已提交
103

104
const std::string kDefaultColumnFamilyName("default");
105

I
Igor Canadi 已提交
106
void DumpRocksDBBuildVersion(Logger * log);
107

S
Stanislau Hlebik 已提交
108 109
struct DBImpl::WriteContext {
  autovector<SuperVersion*> superversions_to_free_;
110
  autovector<MemTable*> memtables_to_free_;
S
Stanislau Hlebik 已提交
111 112 113 114 115

  ~WriteContext() {
    for (auto& sv : superversions_to_free_) {
      delete sv;
    }
116 117 118
    for (auto& m : memtables_to_free_) {
      delete m;
    }
S
Stanislau Hlebik 已提交
119 120 121
  }
};

J
jorlow@chromium.org 已提交
122 123 124
Options SanitizeOptions(const std::string& dbname,
                        const InternalKeyComparator* icmp,
                        const Options& src) {
125
  auto db_options = SanitizeOptions(dbname, DBOptions(src));
126
  auto cf_options = SanitizeOptions(db_options, icmp, ColumnFamilyOptions(src));
127 128 129 130 131
  return Options(db_options, cf_options);
}

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

133 134
  // result.max_open_files means an "infinite" open files.
  if (result.max_open_files != -1) {
I
Igor Canadi 已提交
135 136 137 138 139
    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);
140
  }
141

142
  if (result.info_log == nullptr) {
143
    Status s = CreateLoggerFromOptions(dbname, result, &result.info_log);
J
jorlow@chromium.org 已提交
144 145
    if (!s.ok()) {
      // No place suitable for logging
146
      result.info_log = nullptr;
J
jorlow@chromium.org 已提交
147 148
    }
  }
149 150 151 152
  result.env->IncBackgroundThreadsIfNeeded(src.max_background_compactions,
                                           Env::Priority::LOW);
  result.env->IncBackgroundThreadsIfNeeded(src.max_background_flushes,
                                           Env::Priority::HIGH);
153

I
Igor Canadi 已提交
154
  if (result.rate_limiter.get() != nullptr) {
155 156 157 158 159
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

160 161 162 163
  if (result.WAL_ttl_seconds > 0 || result.WAL_size_limit_MB > 0) {
    result.recycle_log_file_num = false;
  }

164 165 166 167
  if (result.wal_dir.empty()) {
    // Use dbname as default
    result.wal_dir = dbname;
  }
168
  if (result.wal_dir.back() == '/') {
I
Igor Canadi 已提交
169
    result.wal_dir = result.wal_dir.substr(0, result.wal_dir.size() - 1);
170
  }
171

172
  if (result.db_paths.size() == 0) {
173
    result.db_paths.emplace_back(dbname, std::numeric_limits<uint64_t>::max());
174 175
  }

176 177 178 179
  if (result.compaction_readahead_size > 0) {
    result.new_table_reader_for_compaction_inputs = true;
  }

J
jorlow@chromium.org 已提交
180 181 182
  return result;
}

183 184
namespace {

185 186
Status SanitizeOptionsByTable(
    const DBOptions& db_opts,
187 188 189
    const std::vector<ColumnFamilyDescriptor>& column_families) {
  Status s;
  for (auto cf : column_families) {
190
    s = cf.options.table_factory->SanitizeOptions(db_opts, cf.options);
191 192 193 194 195 196 197
    if (!s.ok()) {
      return s;
    }
  }
  return Status::OK();
}

198
CompressionType GetCompressionFlush(const ImmutableCFOptions& ioptions) {
199 200 201 202 203 204
  // 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;

205
  if (ioptions.compaction_style == kCompactionStyleUniversal) {
206
    can_compress =
207
        (ioptions.compaction_options_universal.compression_size_percent < 0);
208 209
  } else {
    // For leveled compress when min_level_to_compress == 0.
210 211
    can_compress = ioptions.compression_per_level.empty() ||
                   ioptions.compression_per_level[0] != kNoCompression;
212 213 214
  }

  if (can_compress) {
215
    return ioptions.compression;
216 217 218 219
  } else {
    return kNoCompression;
  }
}
I
Igor Canadi 已提交
220

221
void DumpSupportInfo(Logger* logger) {
I
Igor Canadi 已提交
222 223 224 225 226 227 228 229
  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());
230 231
  Log(InfoLogLevel::INFO_LEVEL, logger, "Fast CRC32 supported: %d",
      crc32c::IsFastCrc32Supported());
I
Igor Canadi 已提交
232 233
}

234
}  // namespace
235

I
Igor Canadi 已提交
236
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname)
J
jorlow@chromium.org 已提交
237
    : env_(options.env),
H
heyongqiang 已提交
238
      dbname_(dbname),
239 240
      db_options_(SanitizeOptions(dbname, options)),
      stats_(db_options_.statistics.get()),
241
      db_lock_(nullptr),
I
Igor Canadi 已提交
242
      mutex_(stats_, env_, DB_MUTEX_WAIT_MICROS, options.use_adaptive_mutex),
I
Igor Canadi 已提交
243
      shutting_down_(false),
J
jorlow@chromium.org 已提交
244
      bg_cv_(&mutex_),
245
      logfile_number_(0),
246
      log_dir_synced_(false),
I
Igor Canadi 已提交
247
      log_empty_(true),
248
      default_cf_handle_(nullptr),
249
      log_sync_cv_(&mutex_),
I
Igor Canadi 已提交
250 251
      total_log_size_(0),
      max_total_in_memory_state_(0),
S
sdong 已提交
252
      is_snapshot_supported_(true),
253
      write_buffer_(options.db_write_buffer_size),
254 255 256 257
      write_thread_(options.enable_write_thread_adaptive_yield
                        ? options.write_thread_max_yield_usec
                        : 0,
                    options.write_thread_slow_yield_usec),
S
sdong 已提交
258 259
      write_controller_(options.delayed_write_rate),
      last_batch_group_size_(0),
260 261
      unscheduled_flushes_(0),
      unscheduled_compactions_(0),
262
      bg_compaction_scheduled_(0),
263
      num_running_compactions_(0),
264
      bg_flush_scheduled_(0),
265
      num_running_flushes_(0),
266
      disable_delete_obsolete_files_(0),
I
Igor Canadi 已提交
267 268 269
      delete_obsolete_files_next_run_(
          options.env->NowMicros() +
          db_options_.delete_obsolete_files_period_micros),
270
      last_stats_dump_time_microsec_(0),
271
      next_job_id_(1),
272
      flush_on_destroy_(false),
273
      env_options_(db_options_),
I
Igor Canadi 已提交
274 275 276
#ifndef ROCKSDB_LITE
      wal_manager_(db_options_, env_options_),
#endif  // ROCKSDB_LITE
I
Igor Canadi 已提交
277
      event_logger_(db_options_.info_log.get()),
278
      bg_work_paused_(0),
279
      refitting_level_(false),
280
      opened_successfully_(false) {
H
heyongqiang 已提交
281
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
282

J
jorlow@chromium.org 已提交
283
  // Reserve ten files or so for other uses and give the rest to TableCache.
284
  // Give a large number for setting of "infinite" open files.
285 286
  const int table_cache_size = (db_options_.max_open_files == -1) ?
        4194304 : db_options_.max_open_files - 10;
287
  table_cache_ =
288
      NewLRUCache(table_cache_size, db_options_.table_cache_numshardbits);
289

290
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
291 292
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
293 294
  column_family_memtables_.reset(
      new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet()));
295

I
Igor Canadi 已提交
296
  DumpRocksDBBuildVersion(db_options_.info_log.get());
297 298
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
299
  DumpSupportInfo(db_options_.info_log.get());
J
jorlow@chromium.org 已提交
300 301
}

302
// Will lock the mutex_,  will wait for completion if wait is true
303
void DBImpl::CancelAllBackgroundWork(bool wait) {
304
  InstrumentedMutexLock l(&mutex_);
305
  shutting_down_.store(true, std::memory_order_release);
306
  bg_cv_.SignalAll();
307 308 309 310
  if (!wait) {
    return;
  }
  // Wait for background work to finish
311
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
312 313
    bg_cv_.Wait();
  }
314 315
}

J
jorlow@chromium.org 已提交
316
DBImpl::~DBImpl() {
317
  mutex_.Lock();
318

319
  if (!shutting_down_.load(std::memory_order_acquire) && flush_on_destroy_) {
320
    for (auto cfd : *versions_->GetColumnFamilySet()) {
321
      if (!cfd->IsDropped() && !cfd->mem()->IsEmpty()) {
322 323 324 325
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
326
        cfd->Unref();
327 328
      }
    }
329
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
330
  }
331 332 333
  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
334 335
  // (to consider: moving all the waiting into CancelAllBackgroundWork(true))
  CancelAllBackgroundWork(false);
336 337 338 339 340 341 342
  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
343
  while (bg_compaction_scheduled_ || bg_flush_scheduled_) {
344 345
    bg_cv_.Wait();
  }
346
  EraseThreadStatusDbInfo();
I
Igor Canadi 已提交
347 348
  flush_scheduler_.Clear();

349 350 351 352 353 354 355 356 357 358 359 360 361
  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 已提交
362 363 364 365 366
  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();
367 368
  }

I
Igor Canadi 已提交
369 370 371 372 373 374 375 376 377 378
  // 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_) {
379
    JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
380
    FindObsoleteFiles(&job_context, true);
381 382

    mutex_.Unlock();
I
Igor Canadi 已提交
383
    // manifest number starting from 2
I
Igor Canadi 已提交
384 385 386
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
387
    }
I
Igor Canadi 已提交
388
    job_context.Clean();
389
    mutex_.Lock();
390 391
  }

392 393 394
  for (auto l : logs_to_free_) {
    delete l;
  }
395 396 397
  for (auto& log : logs_) {
    log.ClearWriter();
  }
398
  logs_.clear();
399

400
  // versions need to be destroyed before table_cache since it can hold
401 402
  // references to table_cache.
  versions_.reset();
403
  mutex_.Unlock();
I
Igor Canadi 已提交
404 405 406
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
407

408
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
409 410 411
}

Status DBImpl::NewDB() {
412
  VersionEdit new_db;
413
  new_db.SetLogNumber(0);
J
jorlow@chromium.org 已提交
414 415 416
  new_db.SetNextFile(2);
  new_db.SetLastSequence(0);

417 418
  Status s;

419 420
  Log(InfoLogLevel::INFO_LEVEL,
      db_options_.info_log, "Creating manifest 1 \n");
J
jorlow@chromium.org 已提交
421 422
  const std::string manifest = DescriptorFileName(dbname_, 1);
  {
423 424
    unique_ptr<WritableFile> file;
    EnvOptions env_options = env_->OptimizeForManifestWrite(env_options_);
S
sdong 已提交
425
    s = NewWritableFile(env_, manifest, &file, env_options);
426 427 428 429 430 431
    if (!s.ok()) {
      return s;
    }
    file->SetPreallocationBlockSize(db_options_.manifest_preallocation_size);
    unique_ptr<WritableFileWriter> file_writer(
        new WritableFileWriter(std::move(file), env_options));
432
    log::Writer log(std::move(file_writer), 0, false);
J
jorlow@chromium.org 已提交
433 434 435
    std::string record;
    new_db.EncodeTo(&record);
    s = log.AddRecord(record);
S
sdong 已提交
436 437 438
    if (s.ok()) {
      s = SyncManifest(env_, &db_options_, log.file());
    }
J
jorlow@chromium.org 已提交
439 440 441
  }
  if (s.ok()) {
    // Make "CURRENT" file that points to the new manifest file.
442
    s = SetCurrentFile(env_, dbname_, 1, directories_.GetDbDir());
J
jorlow@chromium.org 已提交
443 444 445 446 447 448 449
  } else {
    env_->DeleteFile(manifest);
  }
  return s;
}

void DBImpl::MaybeIgnoreError(Status* s) const {
450
  if (s->ok() || db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
451 452
    // No change needed
  } else {
453 454
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "Ignoring error %s", s->ToString().c_str());
J
jorlow@chromium.org 已提交
455 456 457 458
    *s = Status::OK();
  }
}

459
const Status DBImpl::CreateArchivalDirectory() {
460 461
  if (db_options_.WAL_ttl_seconds > 0 || db_options_.WAL_size_limit_MB > 0) {
    std::string archivalPath = ArchivalDirectory(db_options_.wal_dir);
462 463 464 465 466
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

467
void DBImpl::PrintStatistics() {
468
  auto dbstats = db_options_.statistics.get();
469
  if (dbstats) {
Y
Yueh-Hsuan Chiang 已提交
470
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
M
Mark Callaghan 已提交
471
        "STATISTICS:\n %s",
472
        dbstats->ToString().c_str());
473 474 475
  }
}

476
void DBImpl::MaybeDumpStats() {
477
  if (db_options_.stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
478 479 480 481

  const uint64_t now_micros = env_->NowMicros();

  if (last_stats_dump_time_microsec_ +
482
      db_options_.stats_dump_period_sec * 1000000
H
Haobo Xu 已提交
483 484 485 486 487 488
      <= 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;
489

490
#ifndef ROCKSDB_LITE
491 492 493
    bool tmp1 = false;
    bool tmp2 = false;
    DBPropertyType cf_property_type =
494
        GetPropertyType(DB::Properties::kCFStats, &tmp1, &tmp2);
495
    DBPropertyType db_property_type =
496
        GetPropertyType(DB::Properties::kDBStats, &tmp1, &tmp2);
H
Haobo Xu 已提交
497
    std::string stats;
498
    {
499
      InstrumentedMutexLock l(&mutex_);
500
      for (auto cfd : *versions_->GetColumnFamilySet()) {
501
        cfd->internal_stats()->GetStringProperty(cf_property_type,
502 503
                                                 DB::Properties::kCFStats,
                                                 &stats);
504
      }
505
      default_cf_internal_stats_->GetStringProperty(db_property_type,
506 507
                                                    DB::Properties::kDBStats,
                                                    &stats);
508
    }
Y
Yueh-Hsuan Chiang 已提交
509
    Log(InfoLogLevel::WARN_LEVEL,
510
        db_options_.info_log, "------- DUMPING STATS -------");
Y
Yueh-Hsuan Chiang 已提交
511
    Log(InfoLogLevel::WARN_LEVEL,
512
        db_options_.info_log, "%s", stats.c_str());
513
#endif  // !ROCKSDB_LITE
514

515
    PrintStatistics();
516 517 518
  }
}

I
Igor Canadi 已提交
519
// * Returns the list of live files in 'sst_live'
I
Igor Canadi 已提交
520
// If it's doing full scan:
I
Igor Canadi 已提交
521 522
// * Returns the list of all files in the filesystem in
// 'full_scan_candidate_files'.
I
Igor Canadi 已提交
523
// Otherwise, gets obsolete files from VersionSet.
I
Igor Canadi 已提交
524 525
// no_full_scan = true -- never do the full scan using GetChildren()
// force = false -- don't force the full scan, except every
526
//  db_options_.delete_obsolete_files_period_micros
I
Igor Canadi 已提交
527
// force = true -- force the full scan
I
Igor Canadi 已提交
528
void DBImpl::FindObsoleteFiles(JobContext* job_context, bool force,
I
Igor Canadi 已提交
529
                               bool no_full_scan) {
D
Dhruba Borthakur 已提交
530 531
  mutex_.AssertHeld();

532
  // if deletion is disabled, do nothing
533
  if (disable_delete_obsolete_files_ > 0) {
534 535 536
    return;
  }

537 538 539 540 541
  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;
542
  } else if (force || db_options_.delete_obsolete_files_period_micros == 0) {
543 544 545
    doing_the_full_scan = true;
  } else {
    const uint64_t now_micros = env_->NowMicros();
I
Igor Canadi 已提交
546
    if (delete_obsolete_files_next_run_ < now_micros) {
547
      doing_the_full_scan = true;
I
Igor Canadi 已提交
548 549
      delete_obsolete_files_next_run_ =
          now_micros + db_options_.delete_obsolete_files_period_micros;
550 551 552
    }
  }

I
Igor Canadi 已提交
553 554
  // don't delete files that might be currently written to from compaction
  // threads
555 556 557
  // 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 已提交
558 559 560 561 562 563 564
  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();
  }

565 566
  // Get obsolete files.  This function will also update the list of
  // pending files in VersionSet().
I
Igor Canadi 已提交
567 568
  versions_->GetObsoleteFiles(&job_context->sst_delete_files,
                              job_context->min_pending_output);
I
Igor Canadi 已提交
569

I
Igor Canadi 已提交
570
  // store the current filenum, lognum, etc
571
  job_context->manifest_file_number = versions_->manifest_file_number();
I
Igor Canadi 已提交
572
  job_context->pending_manifest_file_number =
573
      versions_->pending_manifest_file_number();
I
Igor Canadi 已提交
574
  job_context->log_number = versions_->MinLogNumber();
575
  job_context->prev_log_number = versions_->prev_log_number();
I
Igor Canadi 已提交
576

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

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

  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();
S
Sage Weil 已提交
616 617 618 619 620 621 622
      if (db_options_.recycle_log_file_num > log_recycle_files.size()) {
        Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
            "adding log %" PRIu64 " to recycle list\n", earliest.number);
        log_recycle_files.push_back(earliest.number);
      } else {
        job_context->log_delete_files.push_back(earliest.number);
      }
623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646
      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();
647 648
}

649
namespace {
I
Igor Canadi 已提交
650 651
bool CompareCandidateFile(const JobContext::CandidateFileInfo& first,
                          const JobContext::CandidateFileInfo& second) {
652 653 654 655 656
  if (first.file_name > second.file_name) {
    return true;
  } else if (first.file_name < second.file_name) {
    return false;
  } else {
657
    return (first.path_id > second.path_id);
658 659 660 661
  }
}
};  // namespace

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

I
Igor Canadi 已提交
670 671 672 673
  // 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 已提交
674 675
    return;
  }
676

677
  // Now, convert live list to an unordered map, WITHOUT mutex held;
678
  // set is slow.
679
  std::unordered_map<uint64_t, const FileDescriptor*> sst_live_map;
I
Igor Canadi 已提交
680
  for (const FileDescriptor& fd : state.sst_live) {
681 682
    sst_live_map[fd.GetNumber()] = &fd;
  }
I
Igor Canadi 已提交
683

I
Igor Canadi 已提交
684
  auto candidate_files = state.full_scan_candidate_files;
I
Igor Canadi 已提交
685 686 687
  candidate_files.reserve(candidate_files.size() +
                          state.sst_delete_files.size() +
                          state.log_delete_files.size());
K
kailiu 已提交
688 689
  // We may ignore the dbname when generating the file names.
  const char* kDumbDbName = "";
690
  for (auto file : state.sst_delete_files) {
691 692 693
    candidate_files.emplace_back(
        MakeTableFileName(kDumbDbName, file->fd.GetNumber()),
        file->fd.GetPathId());
694
    delete file;
I
Igor Canadi 已提交
695 696
  }

K
kailiu 已提交
697 698
  for (auto file_num : state.log_delete_files) {
    if (file_num > 0) {
699 700
      candidate_files.emplace_back(LogFileName(kDumbDbName, file_num).substr(1),
                                   0);
I
Igor Canadi 已提交
701 702
    }
  }
703

K
kailiu 已提交
704
  // dedup state.candidate_files so we don't try to delete the same
I
Igor Canadi 已提交
705
  // file twice
706
  sort(candidate_files.begin(), candidate_files.end(), CompareCandidateFile);
707 708
  candidate_files.erase(unique(candidate_files.begin(), candidate_files.end()),
                        candidate_files.end());
J
jorlow@chromium.org 已提交
709

710
  std::vector<std::string> old_info_log_files;
711
  InfoLogPrefix info_log_prefix(!db_options_.db_log_dir.empty(), dbname_);
712 713 714
  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 已提交
715 716 717
    uint64_t number;
    FileType type;
    // Ignore file if we cannot recognize it.
718
    if (!ParseFileName(to_delete, &number, info_log_prefix.prefix, &type)) {
K
kailiu 已提交
719 720
      continue;
    }
J
jorlow@chromium.org 已提交
721

K
kailiu 已提交
722 723 724 725 726 727 728 729
    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'
730
        // (can happen during manifest roll)
K
kailiu 已提交
731 732 733
        keep = (number >= state.manifest_file_number);
        break;
      case kTableFile:
I
Igor Canadi 已提交
734 735 736 737
        // 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 已提交
738 739 740
        break;
      case kTempFile:
        // Any temp files that are currently being written to must
741 742 743 744
        // 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
745 746 747
        //
        // TODO(yhchiang): carefully modify the third condition to safely
        //                 remove the temp options files.
748
        keep = (sst_live_map.find(number) != sst_live_map.end()) ||
749 750
               (number == state.pending_manifest_file_number) ||
               (to_delete.find(kOptionsFileNamePrefix) != std::string::npos);
K
kailiu 已提交
751 752 753 754
        break;
      case kInfoLogFile:
        keep = true;
        if (number != 0) {
755
          old_info_log_files.push_back(to_delete);
J
jorlow@chromium.org 已提交
756
        }
K
kailiu 已提交
757 758 759 760 761
        break;
      case kCurrentFile:
      case kDBLockFile:
      case kIdentityFile:
      case kMetaDatabase:
762
      case kOptionsFile:
K
kailiu 已提交
763 764 765 766 767 768 769 770
        keep = true;
        break;
    }

    if (keep) {
      continue;
    }

771
    std::string fname;
K
kailiu 已提交
772 773
    if (type == kTableFile) {
      // evict from cache
774
      TableCache::Evict(table_cache_.get(), number);
775
      fname = TableFileName(db_options_.db_paths, number, path_id);
776
    } else {
777 778
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
779
    }
780

781
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
782
    if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 ||
783
                              db_options_.WAL_size_limit_MB > 0)) {
I
Igor Canadi 已提交
784
      wal_manager_.ArchiveWALFile(fname, number);
785 786 787
      continue;
    }
#endif  // !ROCKSDB_LITE
I
Islam AbdelRahman 已提交
788
    Status file_deletion_status;
789 790
    if (type == kTableFile && path_id == 0) {
      file_deletion_status = DeleteOrMoveToTrash(&db_options_, fname);
I
Islam AbdelRahman 已提交
791 792 793
    } else {
      file_deletion_status = env_->DeleteFile(fname);
    }
794
    if (file_deletion_status.ok()) {
795
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
796
          "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", state.job_id,
797 798
          fname.c_str(), type, number,
          file_deletion_status.ToString().c_str());
799 800 801 802 803 804
    } 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());
805 806 807 808 809 810 811
    } 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) {
812 813 814 815
      EventHelpers::LogAndNotifyTableFileDeletion(
          &event_logger_, state.job_id, number, fname,
          file_deletion_status, GetName(),
          db_options_.listeners);
J
jorlow@chromium.org 已提交
816 817
    }
  }
H
heyongqiang 已提交
818

819
  // Delete old info log files.
820
  size_t old_info_log_file_count = old_info_log_files.size();
821
  if (old_info_log_file_count >= db_options_.keep_log_file_num) {
822
    std::sort(old_info_log_files.begin(), old_info_log_files.end());
823
    size_t end = old_info_log_file_count - db_options_.keep_log_file_num;
824
    for (unsigned int i = 0; i <= end; i++) {
825
      std::string& to_delete = old_info_log_files.at(i);
826 827
      std::string full_path_to_delete = (db_options_.db_log_dir.empty() ?
           dbname_ : db_options_.db_log_dir) + "/" + to_delete;
828
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
829
          "[JOB %d] Delete info log file %s\n", state.job_id,
830 831
          full_path_to_delete.c_str());
      Status s = env_->DeleteFile(full_path_to_delete);
832
      if (!s.ok()) {
833 834 835 836 837 838 839 840 841 842
        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());
        }
843
      }
H
heyongqiang 已提交
844 845
    }
  }
I
Igor Canadi 已提交
846 847 848
#ifndef ROCKSDB_LITE
  wal_manager_.PurgeObsoleteWALFiles();
#endif  // ROCKSDB_LITE
849
  LogFlush(db_options_.info_log);
D
Dhruba Borthakur 已提交
850 851 852 853
}

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
854
  JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
855
  FindObsoleteFiles(&job_context, true);
856 857

  mutex_.Unlock();
I
Igor Canadi 已提交
858 859
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
860
  }
I
Igor Canadi 已提交
861
  job_context.Clean();
862
  mutex_.Lock();
863 864
}

865 866 867 868 869 870 871 872 873 874 875 876 877 878 879 880
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 已提交
881

882 883 884 885 886 887 888 889 890
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_);
891 892 893
    if (!s.ok()) {
      return s;
    }
894
  }
895

896 897 898 899 900 901 902 903
  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);
904 905 906
      if (!s.ok()) {
        return s;
      }
907
      data_dirs_.emplace_back(path_directory.release());
908
    }
909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927
  }
  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();
928

929 930 931 932 933
  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);
934 935 936 937
    if (!s.ok()) {
      return s;
    }

938
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
939 940 941
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
942

A
agiardullo 已提交
943 944
    s = env_->FileExists(CurrentFileName(dbname_));
    if (s.IsNotFound()) {
945
      if (db_options_.create_if_missing) {
946
        s = NewDB();
947
        is_new_db = true;
948 949 950 951 952 953
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
954
      }
A
agiardullo 已提交
955
    } else if (s.ok()) {
956
      if (db_options_.error_if_exists) {
957 958 959
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
A
agiardullo 已提交
960 961 962 963
    } else {
      // Unexpected error reading file
      assert(s.IsIOError());
      return s;
J
jorlow@chromium.org 已提交
964
    }
M
Mayank Agarwal 已提交
965
    // Check for the IDENTITY file and create it if not there
A
agiardullo 已提交
966 967
    s = env_->FileExists(IdentityFileName(dbname_));
    if (s.IsNotFound()) {
M
Mayank Agarwal 已提交
968 969 970 971
      s = SetIdentityFile(env_, dbname_);
      if (!s.ok()) {
        return s;
      }
A
agiardullo 已提交
972 973 974
    } else if (!s.ok()) {
      assert(s.IsIOError());
      return s;
M
Mayank Agarwal 已提交
975
    }
J
jorlow@chromium.org 已提交
976 977
  }

978
  Status s = versions_->Recover(column_families, read_only);
979
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
980 981
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
982
  if (s.ok()) {
A
agiardullo 已提交
983
    SequenceNumber max_sequence(kMaxSequenceNumber);
984 985
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
986
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
987 988
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
989 990 991 992 993

    // 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).
    //
994
    // Note that prev_log_number() is no longer used, but we pay
995
    // attention to it in case we are recovering a database
996
    // produced by an older version of rocksdb.
997
    const uint64_t min_log = versions_->MinLogNumber();
998
    const uint64_t prev_log = versions_->prev_log_number();
999
    std::vector<std::string> filenames;
1000
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
1001 1002
    if (!s.ok()) {
      return s;
1003
    }
K
kailiu 已提交
1004

1005 1006
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
1007 1008
      uint64_t number;
      FileType type;
1009 1010 1011 1012 1013 1014 1015 1016 1017
      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);
        }
1018
      }
J
jorlow@chromium.org 已提交
1019
    }
1020

H
heyongqiang 已提交
1021 1022 1023 1024 1025 1026
    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 已提交
1027 1028 1029 1030 1031 1032 1033
    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 已提交
1034 1035
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 kMaxSequenceNumber);
S
Stanislau Hlebik 已提交
1036 1037
        }
      }
1038
    }
L
Lei Jin 已提交
1039
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
1040 1041
  }

L
Lei Jin 已提交
1042 1043
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
1044
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
1045 1046 1047
    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 已提交
1048 1049
  }

J
jorlow@chromium.org 已提交
1050 1051 1052
  return s;
}

S
Stanislau Hlebik 已提交
1053 1054 1055
// 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 已提交
1056 1057
  struct LogReporter : public log::Reader::Reporter {
    Env* env;
1058
    Logger* info_log;
J
jorlow@chromium.org 已提交
1059
    const char* fname;
1060
    Status* status;  // nullptr if db_options_.paranoid_checks==false
I
Igor Sugak 已提交
1061
    virtual void Corruption(size_t bytes, const Status& s) override {
1062 1063
      Log(InfoLogLevel::WARN_LEVEL,
          info_log, "%s%s: dropping %d bytes; %s",
1064
          (this->status == nullptr ? "(ignoring error) " : ""),
J
jorlow@chromium.org 已提交
1065
          fname, static_cast<int>(bytes), s.ToString().c_str());
K
krad 已提交
1066 1067 1068
      if (this->status != nullptr && this->status->ok()) {
        *this->status = s;
      }
J
jorlow@chromium.org 已提交
1069 1070 1071 1072
    }
  };

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
1073
  Status status;
1074
  std::unordered_map<int, VersionEdit> version_edits;
1075
  // no need to refcount because iteration is under mutex
1076 1077
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
1078 1079
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
1080
  }
1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092
  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 已提交
1093

K
krad 已提交
1094
  bool continue_replay_log = true;
S
Stanislau Hlebik 已提交
1095 1096 1097 1098
  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.
1099
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
1100 1101
    // Open the log file
    std::string fname = LogFileName(db_options_.wal_dir, log_number);
1102 1103 1104 1105
    unique_ptr<SequentialFileReader> file_reader;
    {
      unique_ptr<SequentialFile> file;
      status = env_->NewSequentialFile(fname, &file, env_options_);
S
Stanislau Hlebik 已提交
1106
      if (!status.ok()) {
1107 1108 1109 1110 1111 1112 1113 1114
        MaybeIgnoreError(&status);
        if (!status.ok()) {
          return status;
        } else {
          // Fail with one log file, but that's ok.
          // Try next one.
          continue;
        }
S
Stanislau Hlebik 已提交
1115
      }
1116
      file_reader.reset(new SequentialFileReader(std::move(file)));
J
jorlow@chromium.org 已提交
1117 1118
    }

S
Stanislau Hlebik 已提交
1119 1120 1121 1122 1123
    // Create the log reader.
    LogReporter reporter;
    reporter.env = env_;
    reporter.info_log = db_options_.info_log.get();
    reporter.fname = fname.c_str();
K
krad 已提交
1124 1125 1126 1127 1128 1129 1130
    if (!db_options_.paranoid_checks ||
        db_options_.wal_recovery_mode ==
            WALRecoveryMode::kSkipAnyCorruptedRecords) {
      reporter.status = nullptr;
    } else {
      reporter.status = &status;
    }
S
Stanislau Hlebik 已提交
1131 1132 1133 1134
    // 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).
1135 1136
    log::Reader reader(db_options_.info_log, std::move(file_reader), &reporter,
                       true /*checksum*/, 0 /*initial_offset*/, log_number);
K
krad 已提交
1137 1138 1139 1140 1141
    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
S
Stanislau Hlebik 已提交
1142 1143 1144 1145
    // Read all the records and add to a memtable
    std::string scratch;
    Slice record;
    WriteBatch batch;
K
krad 已提交
1146 1147 1148 1149 1150 1151 1152 1153 1154 1155

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

1156 1157 1158 1159
    while (
        continue_replay_log &&
        reader.ReadRecord(&record, &scratch, db_options_.wal_recovery_mode) &&
        status.ok()) {
S
Stanislau Hlebik 已提交
1160 1161 1162 1163 1164 1165 1166
      if (record.size() < 12) {
        reporter.Corruption(record.size(),
                            Status::Corruption("log record too small"));
        continue;
      }
      WriteBatchInternal::SetContents(&batch, record);

1167
#ifndef ROCKSDB_LITE
1168
      if (db_options_.wal_filter != nullptr) {
1169 1170 1171
        WriteBatch new_batch;
        bool batch_changed = false;

1172
        WalFilter::WalProcessingOption wal_processing_option =
S
sdong 已提交
1173 1174
            db_options_.wal_filter->LogRecord(batch, &new_batch,
                                              &batch_changed);
1175

1176
        switch (wal_processing_option) {
S
sdong 已提交
1177 1178 1179 1180 1181
          case WalFilter::WalProcessingOption::kContinueProcessing:
            // do nothing, proceeed normally
            break;
          case WalFilter::WalProcessingOption::kIgnoreCurrentRecord:
            // skip current record
1182
            continue;
S
sdong 已提交
1183 1184 1185
          case WalFilter::WalProcessingOption::kStopReplay:
            // skip current record and stop replay
            continue_replay_log = false;
1186
            continue;
S
sdong 已提交
1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209
          case WalFilter::WalProcessingOption::kCorruptedRecord: {
            status = Status::Corruption("Corruption reported by Wal Filter ",
                                        db_options_.wal_filter->Name());
            MaybeIgnoreError(&status);
            if (!status.ok()) {
              reporter.Corruption(record.size(), status);
              continue;
            }
            break;
          }
          default: {
            assert(false);  // unhandled case
            status = Status::NotSupported(
                "Unknown WalProcessingOption returned"
                " by Wal Filter ",
                db_options_.wal_filter->Name());
            MaybeIgnoreError(&status);
            if (!status.ok()) {
              return status;
            } else {
              // Ignore the error with current record processing.
              continue;
            }
1210 1211 1212 1213 1214 1215 1216 1217 1218
          }
        }

        if (batch_changed) {
          // Make sure that the count in the new batch is
          // within the orignal count.
          int new_count = WriteBatchInternal::Count(&new_batch);
          int original_count = WriteBatchInternal::Count(&batch);
          if (new_count > original_count) {
1219
            Log(InfoLogLevel::FATAL_LEVEL, db_options_.info_log,
S
sdong 已提交
1220 1221 1222 1223 1224 1225 1226 1227 1228 1229
                "Recovering log #%" PRIu64
                " mode %d log filter %s returned "
                "more records (%d) than original (%d) which is not allowed. "
                "Aborting recovery.",
                log_number, db_options_.wal_recovery_mode,
                db_options_.wal_filter->Name(), new_count, original_count);
            status = Status::NotSupported(
                "More than original # of records "
                "returned by Wal Filter ",
                db_options_.wal_filter->Name());
1230
            return status;
1231 1232 1233
          }
          // Set the same sequence number in the new_batch
          // as the original batch.
S
sdong 已提交
1234 1235
          WriteBatchInternal::SetSequence(&new_batch,
                                          WriteBatchInternal::Sequence(&batch));
1236
          batch = new_batch;
1237 1238
        }
      }
S
sdong 已提交
1239
#endif  // ROCKSDB_LITE
1240

S
Stanislau Hlebik 已提交
1241 1242 1243 1244 1245
      // 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
1246 1247 1248
      status =
          WriteBatchInternal::InsertInto(&batch, column_family_memtables_.get(),
                                         &flush_scheduler_, true, log_number);
S
Stanislau Hlebik 已提交
1249 1250 1251

      MaybeIgnoreError(&status);
      if (!status.ok()) {
1252 1253 1254 1255
        // 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 已提交
1256
      }
1257

S
Stanislau Hlebik 已提交
1258 1259
      const SequenceNumber last_seq = WriteBatchInternal::Sequence(&batch) +
                                      WriteBatchInternal::Count(&batch) - 1;
A
agiardullo 已提交
1260
      if ((*max_sequence == kMaxSequenceNumber) || (last_seq > *max_sequence)) {
S
Stanislau Hlebik 已提交
1261 1262 1263 1264
        *max_sequence = last_seq;
      }

      if (!read_only) {
I
Igor Canadi 已提交
1265 1266 1267 1268
        // 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;

1269
        while ((cfd = flush_scheduler_.TakeNextColumnFamily()) != nullptr) {
I
Igor Canadi 已提交
1270 1271 1272 1273 1274 1275 1276
          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;
1277
          status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
I
Igor Canadi 已提交
1278 1279 1280 1281
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
1282
          }
A
agiardullo 已提交
1283 1284 1285

          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions(),
                                 *max_sequence);
1286
        }
J
jorlow@chromium.org 已提交
1287 1288 1289
      }
    }

1290
    if (!status.ok()) {
K
krad 已提交
1291
      if (db_options_.wal_recovery_mode ==
1292 1293 1294 1295
             WALRecoveryMode::kSkipAnyCorruptedRecords) {
        // We should ignore all errors unconditionally
        status = Status::OK();
      } else if (db_options_.wal_recovery_mode ==
K
krad 已提交
1296 1297 1298 1299 1300 1301 1302 1303
                 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);
1304 1305 1306 1307 1308
      } else {
        assert(db_options_.wal_recovery_mode ==
                  WALRecoveryMode::kTolerateCorruptedTailRecords
               || db_options_.wal_recovery_mode ==
                  WALRecoveryMode::kAbsoluteConsistency);
K
krad 已提交
1309 1310
        return status;
      }
1311 1312
    }

I
Igor Canadi 已提交
1313
    flush_scheduler_.Clear();
A
agiardullo 已提交
1314 1315
    if ((*max_sequence != kMaxSequenceNumber) &&
        (versions_->LastSequence() < *max_sequence)) {
S
Stanislau Hlebik 已提交
1316 1317
      versions_->SetLastSequence(*max_sequence);
    }
1318 1319
  }

1320
  if (!read_only) {
1321 1322
    // 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 已提交
1323
    auto max_log_number = log_numbers.back();
1324
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1325
      auto iter = version_edits.find(cfd->GetID());
1326 1327 1328
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
1329
      if (cfd->GetLogNumber() > max_log_number) {
1330
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
1331
        // from all logs. Memtable has to be empty because
1332
        // we filter the updates based on log_number
1333
        // (in WriteBatch::InsertInto)
1334 1335 1336 1337 1338 1339 1340
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1341
        status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
1342 1343 1344 1345
        if (!status.ok()) {
          // Recovery failed
          break;
        }
A
agiardullo 已提交
1346 1347 1348

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

1351
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
1352
      // writing log_number in the manifest means that any log file
1353 1354
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
1355 1356 1357
      // 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);
1358 1359 1360
      // 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 已提交
1361
      // log number
1362
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
1363 1364
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
1365
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1366 1367
        // Recovery failed
        break;
1368 1369
      }
    }
1370
  }
I
Igor Canadi 已提交
1371

1372 1373 1374
  event_logger_.Log() << "job" << job_id << "event"
                      << "recovery_finished";

J
jorlow@chromium.org 已提交
1375 1376 1377
  return status;
}

1378 1379
Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
                                           MemTable* mem, VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1380
  mutex_.AssertHeld();
1381
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1382
  FileMetaData meta;
1383
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1384 1385
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1386 1387
  ReadOptions ro;
  ro.total_order_seek = true;
1388
  Arena arena;
1389
  Status s;
1390
  TableProperties table_properties;
1391
  {
1392
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
1393 1394 1395
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1396
        cfd->GetName().c_str(), meta.fd.GetNumber());
1397

1398 1399
    bool paranoid_file_checks =
        cfd->GetLatestMutableCFOptions()->paranoid_file_checks;
1400 1401
    {
      mutex_.Unlock();
1402
      TableFileCreationInfo info;
1403 1404 1405 1406 1407

      SequenceNumber earliest_write_conflict_snapshot;
      std::vector<SequenceNumber> snapshot_seqs =
          snapshots_.GetAll(&earliest_write_conflict_snapshot);

1408 1409
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
1410
          iter.get(), &meta, cfd->internal_comparator(),
1411 1412 1413
          cfd->int_tbl_prop_collector_factories(), cfd->GetID(), snapshot_seqs,
          earliest_write_conflict_snapshot,
          GetCompressionFlush(*cfd->ioptions()),
1414 1415
          cfd->ioptions()->compression_opts, paranoid_file_checks,
          cfd->internal_stats(), Env::IO_HIGH, &info.table_properties);
1416
      LogFlush(db_options_.info_log);
1417 1418 1419 1420 1421 1422 1423 1424
      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()) {
1425 1426 1427 1428 1429 1430 1431 1432 1433
        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);
1434
      }
1435 1436
      mutex_.Lock();
    }
1437
  }
I
Igor Canadi 已提交
1438
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1439 1440 1441 1442

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1443
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1444 1445
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
1446 1447
                  meta.smallest_seqno, meta.largest_seqno,
                  meta.marked_for_compaction);
1448 1449
  }

L
Lei Jin 已提交
1450
  InternalStats::CompactionStats stats(1);
1451
  stats.micros = env_->NowMicros() - start_micros;
1452
  stats.bytes_written = meta.fd.GetFileSize();
1453
  stats.num_output_files = 1;
1454
  cfd->internal_stats()->AddCompactionStats(level, stats);
1455 1456
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1457
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1458 1459 1460
  return s;
}

1461 1462
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
1463
    bool* made_progress, JobContext* job_context, LogBuffer* log_buffer) {
1464
  mutex_.AssertHeld();
1465
  assert(cfd->imm()->NumNotFlushed() != 0);
1466
  assert(cfd->imm()->IsFlushPending());
1467

1468 1469 1470 1471 1472 1473 1474 1475 1476 1477
  SequenceNumber earliest_write_conflict_snapshot;
  std::vector<SequenceNumber> snapshot_seqs =
      snapshots_.GetAll(&earliest_write_conflict_snapshot);

  FlushJob flush_job(
      dbname_, cfd, db_options_, mutable_cf_options, env_options_,
      versions_.get(), &mutex_, &shutting_down_, snapshot_seqs,
      earliest_write_conflict_snapshot, job_context, log_buffer,
      directories_.GetDbDir(), directories_.GetDataDir(0U),
      GetCompressionFlush(*cfd->ioptions()), stats_, &event_logger_);
1478

1479
  FileMetaData file_meta;
1480 1481 1482 1483 1484 1485 1486

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

  if (s.ok()) {
I
Igor Canadi 已提交
1490 1491
    InstallSuperVersionAndScheduleWorkWrapper(cfd, job_context,
                                              mutable_cf_options);
1492 1493
    if (made_progress) {
      *made_progress = 1;
1494
    }
S
sdong 已提交
1495
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1496
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1497
                cfd->current()->storage_info()->LevelSummary(&tmp));
J
jorlow@chromium.org 已提交
1498
  }
1499

1500
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1501 1502 1503 1504 1505
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1506
  RecordFlushIOStats();
1507 1508 1509
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1510
    NotifyOnFlushCompleted(cfd, &file_meta, mutable_cf_options,
1511
                           job_context->job_id, flush_job.GetTableProperties());
1512 1513
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1514 1515 1516
  return s;
}

1517 1518 1519 1520
void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd,
                                    FileMetaData* file_meta,
                                    const MutableCFOptions& mutable_cf_options,
                                    int job_id, TableProperties prop) {
1521
#ifndef ROCKSDB_LITE
1522
  if (db_options_.listeners.size() == 0U) {
1523 1524
    return;
  }
1525 1526 1527 1528
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
1529
  bool triggered_writes_slowdown =
1530
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1531
       mutable_cf_options.level0_slowdown_writes_trigger);
1532
  bool triggered_writes_stop =
1533
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1534
       mutable_cf_options.level0_stop_writes_trigger);
1535 1536
  // release lock while notifying events
  mutex_.Unlock();
1537
  {
1538 1539 1540 1541 1542
    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,
1543
                                       file_meta->fd.GetNumber());
1544
    info.thread_id = env_->GetThreadID();
1545 1546 1547
    info.job_id = job_id;
    info.triggered_writes_slowdown = triggered_writes_slowdown;
    info.triggered_writes_stop = triggered_writes_stop;
1548 1549
    info.smallest_seqno = file_meta->smallest_seqno;
    info.largest_seqno = file_meta->largest_seqno;
1550
    info.table_properties = prop;
1551
    for (auto listener : db_options_.listeners) {
1552
      listener->OnFlushCompleted(this, info);
1553 1554
    }
  }
1555 1556 1557
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
1558
#endif  // ROCKSDB_LITE
1559 1560
}

1561 1562 1563 1564
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()) {
1565 1566 1567
    return Status::InvalidArgument("Invalid target path ID");
  }

1568 1569
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
1570
  bool exclusive = options.exclusive_manual_compaction;
I
Igor Canadi 已提交
1571 1572

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1573
  if (!s.ok()) {
1574
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1575 1576 1577
    return s;
  }

I
Igor Canadi 已提交
1578
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1579
  {
1580
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
1581
    Version* base = cfd->current();
1582 1583
    for (int level = 1; level < base->storage_info()->num_non_empty_levels();
         level++) {
S
sdong 已提交
1584
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1585 1586 1587 1588
        max_level_with_files = level;
      }
    }
  }
1589

1590
  int final_output_level = 0;
1591 1592 1593 1594
  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
1595
                            cfd->NumberLevels() - 1, options.target_path_id,
1596
                            begin, end, exclusive);
1597
    final_output_level = cfd->NumberLevels() - 1;
1598 1599
  } else {
    for (int level = 0; level <= max_level_with_files; level++) {
1600
      int output_level;
A
Andres Notzli 已提交
1601
      // in case the compaction is universal or if we're compacting the
1602 1603 1604 1605
      // 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 ||
1606 1607 1608
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
        output_level = level;
      } else if (level == max_level_with_files && level > 0) {
1609 1610 1611 1612 1613 1614 1615
        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 已提交
1616
                   cfd->ioptions()->compaction_filter_factory == nullptr) {
A
Andres Notzli 已提交
1617 1618
          // Skip bottommost level compaction since we don't have a compaction
          // filter
1619 1620
          continue;
        }
1621
        output_level = level;
1622
      } else {
1623
        output_level = level + 1;
1624 1625 1626 1627 1628
        if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
            cfd->ioptions()->level_compaction_dynamic_level_bytes &&
            level == 0) {
          output_level = ColumnFamilyData::kCompactToBaseLevel;
        }
1629
      }
1630
      s = RunManualCompaction(cfd, level, output_level, options.target_path_id,
1631
                              begin, end, exclusive);
1632 1633 1634
      if (!s.ok()) {
        break;
      }
1635 1636 1637 1638 1639
      if (output_level == ColumnFamilyData::kCompactToBaseLevel) {
        final_output_level = cfd->NumberLevels() - 1;
      } else if (output_level > final_output_level) {
        final_output_level = output_level;
      }
1640 1641
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
1642
    }
G
Gabor Cselle 已提交
1643
  }
1644 1645 1646 1647
  if (!s.ok()) {
    LogFlush(db_options_.info_log);
    return s;
  }
1648

1649
  if (options.change_level) {
1650 1651 1652 1653 1654 1655 1656
    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();
1657
  }
1658
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1659

1660
  {
1661
    InstrumentedMutexLock l(&mutex_);
1662 1663 1664 1665 1666
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1667
  return s;
1668 1669
}

1670 1671 1672 1673 1674
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 已提交
1675 1676 1677 1678
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1679 1680 1681 1682 1683 1684
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);
1685 1686 1687 1688 1689 1690 1691 1692 1693 1694 1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708 1709

  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());
1710
  }  // release the mutex
1711 1712 1713 1714 1715 1716 1717 1718 1719 1720

  // 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()) {
1721
      // no mutex is locked here.  No need to Unlock() and Lock() here.
1722 1723 1724 1725 1726
      PurgeObsoleteFiles(job_context);
    }
    job_context.Clean();
  }

1727
  return s;
I
Igor Canadi 已提交
1728
#endif  // ROCKSDB_LITE
1729 1730
}

I
Igor Canadi 已提交
1731
#ifndef ROCKSDB_LITE
1732 1733 1734
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
1735 1736
    const int output_level, int output_path_id, JobContext* job_context,
    LogBuffer* log_buffer) {
1737 1738 1739 1740 1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767 1768
  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;
  }

1769
  std::vector<CompactionInputFiles> input_files;
1770 1771 1772 1773 1774 1775 1776 1777 1778 1779 1780 1781 1782 1783 1784 1785 1786 1787 1788 1789
  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(
1790 1791
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
1792 1793 1794
  if (!c) {
    return Status::Aborted("Another Level 0 compaction is running");
  }
1795 1796
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
1797
  assert(!c->deletion_compaction());
1798

1799 1800 1801 1802
  SequenceNumber earliest_write_conflict_snapshot;
  std::vector<SequenceNumber> snapshot_seqs =
      snapshots_.GetAll(&earliest_write_conflict_snapshot);

I
Igor Canadi 已提交
1803
  assert(is_snapshot_supported_ || snapshots_.empty());
1804
  CompactionJob compaction_job(
I
Igor Canadi 已提交
1805 1806
      job_context->job_id, c.get(), db_options_, env_options_, versions_.get(),
      &shutting_down_, log_buffer, directories_.GetDbDir(),
1807 1808
      directories_.GetDataDir(c->output_path_id()), stats_, snapshot_seqs,
      earliest_write_conflict_snapshot, table_cache_, &event_logger_,
1809 1810
      c->mutable_cf_options()->paranoid_file_checks,
      c->mutable_cf_options()->compaction_measure_io_stats, dbname_,
1811 1812 1813 1814 1815 1816 1817 1818 1819 1820 1821 1822 1823
      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.
1824 1825 1826
  compaction_job.Prepare();

  mutex_.Unlock();
1827 1828
  TEST_SYNC_POINT("CompactFilesImpl:0");
  TEST_SYNC_POINT("CompactFilesImpl:1");
1829
  compaction_job.Run();
1830
  mutex_.Lock();
1831

1832
  Status status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
1833
  if (status.ok()) {
I
Igor Canadi 已提交
1834 1835
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
1836 1837 1838 1839 1840 1841 1842 1843 1844
  }
  c->ReleaseCompactionFiles(s);
  c.reset();

  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
1845 1846
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] [JOB %d] Compaction error: %s",
1847
        c->column_family_data()->GetName().c_str(), job_context->job_id,
1848 1849 1850 1851 1852 1853 1854
        status.ToString().c_str());
    if (db_options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }

  bg_compaction_scheduled_--;
1855 1856 1857
  if (bg_compaction_scheduled_ == 0) {
    bg_cv_.SignalAll();
  }
1858 1859 1860

  return status;
}
I
Igor Canadi 已提交
1861
#endif  // ROCKSDB_LITE
1862

1863 1864 1865 1866 1867 1868 1869 1870 1871 1872 1873
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_);
1874 1875 1876
  if (bg_work_paused_ == 0) {
    return Status::InvalidArgument();
  }
1877 1878 1879 1880 1881 1882 1883 1884
  assert(bg_work_paused_ > 0);
  bg_work_paused_--;
  if (bg_work_paused_ == 0) {
    MaybeScheduleFlushOrCompaction();
  }
  return Status::OK();
}

O
Ori Bernstein 已提交
1885
void DBImpl::NotifyOnCompactionCompleted(
1886 1887
    ColumnFamilyData* cfd, Compaction *c, const Status &st,
    const CompactionJobStats& compaction_job_stats,
1888
    const int job_id) {
O
Ori Bernstein 已提交
1889
#ifndef ROCKSDB_LITE
1890
  if (db_options_.listeners.size() == 0U) {
O
Ori Bernstein 已提交
1891 1892 1893 1894 1895 1896 1897 1898
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  // release lock while notifying events
  mutex_.Unlock();
1899 1900 1901 1902
  {
    CompactionJobInfo info;
    info.cf_name = cfd->GetName();
    info.status = st;
1903
    info.thread_id = env_->GetThreadID();
1904 1905
    info.job_id = job_id;
    info.base_input_level = c->start_level();
1906
    info.output_level = c->output_level();
1907
    info.stats = compaction_job_stats;
1908
    info.table_properties = c->GetOutputTableProperties();
1909
    info.compaction_reason = c->compaction_reason();
1910 1911
    for (size_t i = 0; i < c->num_input_levels(); ++i) {
      for (const auto fmd : *c->inputs(i)) {
1912 1913 1914 1915 1916
        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;
1917
          auto s = cfd->current()->GetTableProperties(&tp, fmd, &fn);
1918 1919 1920 1921
          if (s.ok()) {
            info.table_properties[fn] = tp;
          }
        }
1922 1923 1924 1925 1926 1927 1928 1929 1930 1931 1932 1933
      }
    }
    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 已提交
1934 1935 1936 1937 1938 1939
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
#endif  // ROCKSDB_LITE
}

1940
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1941
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1942 1943 1944
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1945 1946
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1947 1948
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1949
        cfd->GetName().c_str());
1950
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1951 1952 1953
  }

  MutableCFOptions new_options;
1954
  Status s;
1955
  Status persist_options_status;
L
Lei Jin 已提交
1956
  {
1957
    InstrumentedMutexLock l(&mutex_);
1958 1959
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1960 1961
      new_options = *cfd->GetLatestMutableCFOptions();
    }
1962 1963 1964 1965 1966 1967 1968 1969
    if (s.ok()) {
      // Persist RocksDB options under the single write thread
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);

      persist_options_status = WriteOptionsFile();

      write_thread_.ExitUnbatched(&w);
1970 1971
    }
  }
L
Lei Jin 已提交
1972

1973 1974
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1975 1976
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1977 1978
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1979
  }
1980
  if (s.ok()) {
1981 1982
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1983 1984
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
1985 1986 1987 1988 1989 1990 1991 1992 1993 1994
    if (!persist_options_status.ok()) {
      if (db_options_.fail_if_options_file_error) {
        s = Status::IOError(
            "SetOptions succeeded, but unable to persist options",
            persist_options_status.ToString());
      }
      Warn(db_options_.info_log,
           "Unable to persist options in SetOptions() -- %s",
           persist_options_status.ToString().c_str());
    }
L
Lei Jin 已提交
1995
  } else {
1996 1997
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1998
  }
S
sdong 已提交
1999
  LogFlush(db_options_.info_log);
2000
  return s;
I
Igor Canadi 已提交
2001
#endif  // ROCKSDB_LITE
2002 2003
}

2004
// return the same level if it cannot be moved
2005 2006
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
2007
  mutex_.AssertHeld();
S
sdong 已提交
2008
  const auto* vstorage = cfd->current()->storage_info();
2009
  int minimum_level = level;
2010
  for (int i = level - 1; i > 0; --i) {
2011
    // stop if level i is not empty
S
sdong 已提交
2012
    if (vstorage->NumLevelFiles(i) > 0) break;
2013
    // stop if level i is too small (cannot fit the level files)
2014
    if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) {
2015 2016
      break;
    }
2017 2018 2019 2020 2021 2022

    minimum_level = i;
  }
  return minimum_level;
}

2023 2024
// REQUIREMENT: block all background work by calling PauseBackgroundWork()
// before calling this function
I
Igor Canadi 已提交
2025 2026
Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
  assert(level < cfd->NumberLevels());
2027 2028 2029
  if (target_level >= cfd->NumberLevels()) {
    return Status::InvalidArgument("Target level exceeds number of levels");
  }
2030

2031 2032 2033 2034
  std::unique_ptr<SuperVersion> superversion_to_free;
  std::unique_ptr<SuperVersion> new_superversion(new SuperVersion());

  Status status;
I
Igor Canadi 已提交
2035

I
Islam AbdelRahman 已提交
2036
  InstrumentedMutexLock guard_lock(&mutex_);
2037 2038 2039

  // only allow one thread refitting
  if (refitting_level_) {
2040 2041
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
L
Lei Jin 已提交
2042
    return Status::NotSupported("another thread is refitting");
2043 2044 2045
  }
  refitting_level_ = true;

2046
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
2047
  // move to a smaller level
2048 2049
  int to_level = target_level;
  if (target_level < 0) {
2050
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
2051
  }
2052

2053 2054 2055 2056 2057 2058 2059 2060 2061 2062 2063 2064 2065 2066 2067
  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) {
2068
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
2069 2070
        "[%s] Before refitting:\n%s", cfd->GetName().c_str(),
        cfd->current()->DebugString().data());
2071

2072
    VersionEdit edit;
I
Igor Canadi 已提交
2073
    edit.SetColumnFamily(cfd->GetID());
2074
    for (const auto& f : vstorage->LevelFiles(level)) {
2075
      edit.DeleteFile(level, f->fd.GetNumber());
2076 2077
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
2078 2079
                   f->smallest_seqno, f->largest_seqno,
                   f->marked_for_compaction);
2080
    }
2081
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
2082 2083
        "[%s] Apply version edit:\n%s", cfd->GetName().c_str(),
        edit.DebugString().data());
2084

2085 2086
    status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                    directories_.GetDbDir());
2087 2088
    superversion_to_free.reset(InstallSuperVersionAndScheduleWork(
        cfd, new_superversion.release(), mutable_cf_options));
2089

2090 2091
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
2092
        status.ToString().data());
2093 2094

    if (status.ok()) {
2095
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
2096 2097
          "[%s] After refitting:\n%s", cfd->GetName().c_str(),
          cfd->current()->DebugString().data());
2098 2099 2100 2101
    }
  }

  refitting_level_ = false;
I
Igor Canadi 已提交
2102

L
Lei Jin 已提交
2103
  return status;
G
Gabor Cselle 已提交
2104 2105
}

2106 2107 2108
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
2109 2110
}

2111
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
2112
  return 0;
2113 2114
}

2115 2116
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
2117
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
2118 2119
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
2120 2121
}

L
Lei Jin 已提交
2122
Status DBImpl::Flush(const FlushOptions& flush_options,
2123 2124
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
2125
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
2126 2127
}

2128 2129 2130 2131 2132 2133 2134 2135 2136 2137 2138 2139 2140 2141 2142 2143 2144 2145 2146 2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158 2159
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;
2160
      logs_to_sync.push_back(log.writer);
2161 2162 2163 2164 2165
    }

    need_log_dir_sync = !log_dir_synced_;
  }

2166
  RecordTick(stats_, WAL_FILE_SYNCED);
2167 2168 2169 2170 2171 2172 2173 2174 2175 2176 2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196 2197
  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) {
2198
      logs_to_free_.push_back(log.ReleaseWriter());
A
Andres Noetzli 已提交
2199
      it = logs_.erase(it);
2200 2201 2202 2203 2204
    } else {
      log.getting_synced = false;
      ++it;
    }
  }
A
Andres Noetzli 已提交
2205
  assert(logs_.empty() || (logs_.size() == 1 && !logs_[0].getting_synced));
2206 2207 2208
  log_sync_cv_.SignalAll();
}

2209
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
2210 2211 2212
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
2213
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
2214
                                   int output_level, uint32_t output_path_id,
2215
                                   const Slice* begin, const Slice* end,
2216
                                   bool exclusive, bool disallow_trivial_move) {
2217 2218
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);
2219

G
Gabor Cselle 已提交
2220
  InternalKey begin_storage, end_storage;
2221
  CompactionArg* ca;
G
Gabor Cselle 已提交
2222

2223 2224
  bool scheduled = false;
  bool manual_conflict = false;
H
hans@chromium.org 已提交
2225
  ManualCompaction manual;
I
Igor Canadi 已提交
2226
  manual.cfd = cfd;
2227 2228
  manual.input_level = input_level;
  manual.output_level = output_level;
2229
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
2230
  manual.done = false;
2231
  manual.in_progress = false;
2232 2233
  manual.incomplete = false;
  manual.exclusive = exclusive;
2234
  manual.disallow_trivial_move = disallow_trivial_move;
2235 2236 2237
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
2238 2239
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2240
    manual.begin = nullptr;
G
Gabor Cselle 已提交
2241
  } else {
2242
    begin_storage.SetMaxPossibleForUserKey(*begin);
G
Gabor Cselle 已提交
2243 2244
    manual.begin = &begin_storage;
  }
2245
  if (end == nullptr ||
2246 2247
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
2248
    manual.end = nullptr;
G
Gabor Cselle 已提交
2249
  } else {
2250
    end_storage.SetMinPossibleForUserKey(*end);
G
Gabor Cselle 已提交
2251 2252 2253
    manual.end = &end_storage;
  }

2254
  InstrumentedMutexLock l(&mutex_);
2255

2256 2257 2258 2259 2260
  // 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.
  //
2261
  // HasPendingManualCompaction() is true when at least one thread is inside
2262
  // RunManualCompaction(), i.e. during that time no other compaction will
2263 2264 2265
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
2266
  // RunManualCompaction() from getting to the second while loop below.
2267 2268 2269
  // However, only one of them will actually schedule compaction, while
  // others will wait on a condition variable until it completes.

2270 2271 2272 2273 2274 2275 2276 2277 2278 2279
  AddManualCompaction(&manual);
  TEST_SYNC_POINT_CALLBACK("DBImpl::RunManualCompaction:NotScheduled", &mutex_);
  if (exclusive) {
    while (bg_compaction_scheduled_ > 0) {
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
          "[%s] Manual compaction waiting for all other scheduled background "
          "compactions to finish",
          cfd->GetName().c_str());
      bg_cv_.Wait();
    }
2280
  }
2281

2282 2283
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
2284
      cfd->GetName().c_str());
2285

2286 2287 2288 2289
  // 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) {
2290 2291
    assert(HasPendingManualCompaction());
    manual_conflict = false;
2292
    if (ShouldntRunManualCompaction(&manual) || (manual.in_progress == true) ||
2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303
        scheduled ||
        ((manual.manual_end = &manual.tmp_storage1)&&(
             (manual.compaction = manual.cfd->CompactRange(
                  *manual.cfd->GetLatestMutableCFOptions(), manual.input_level,
                  manual.output_level, manual.output_path_id, manual.begin,
                  manual.end, &manual.manual_end, &manual_conflict)) ==
             nullptr) &&
         manual_conflict)) {
      // exclusive manual compactions should not see a conflict during
      // CompactRange
      assert(!exclusive || !manual_conflict);
2304
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
2305
      bg_cv_.Wait();
2306 2307 2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319 2320
      if (scheduled && manual.incomplete == true) {
        assert(!manual.in_progress);
        scheduled = false;
        manual.incomplete = false;
      }
    } else if (!scheduled) {
      if (manual.compaction == nullptr) {
        manual.done = true;
        bg_cv_.SignalAll();
        continue;
      }
      ca = new CompactionArg;
      ca->db = this;
      ca->m = &manual;
      manual.incomplete = false;
2321
      bg_compaction_scheduled_++;
2322 2323 2324
      env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this,
                     &DBImpl::UnscheduleCallback);
      scheduled = true;
G
Gabor Cselle 已提交
2325
    }
H
hans@chromium.org 已提交
2326
  }
2327

2328
  assert(!manual.in_progress);
2329 2330
  assert(HasPendingManualCompaction());
  RemoveManualCompaction(&manual);
2331
  bg_cv_.SignalAll();
L
Lei Jin 已提交
2332
  return manual.status;
J
jorlow@chromium.org 已提交
2333 2334
}

2335 2336 2337 2338 2339 2340 2341 2342 2343 2344 2345 2346 2347 2348 2349 2350 2351
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);
}

2352
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
2353
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
2354 2355 2356
  Status s;
  {
    WriteContext context;
2357
    InstrumentedMutexLock guard_lock(&mutex_);
2358

2359
    if (cfd->imm()->NumNotFlushed() == 0 && cfd->mem()->IsEmpty()) {
2360 2361 2362 2363
      // Nothing to flush
      return Status::OK();
    }

2364 2365
    WriteThread::Writer w;
    write_thread_.EnterUnbatched(&w, &mutex_);
S
Stanislau Hlebik 已提交
2366

I
Igor Canadi 已提交
2367
    // SwitchMemtable() will release and reacquire mutex
S
Stanislau Hlebik 已提交
2368
    // during execution
I
Igor Canadi 已提交
2369
    s = SwitchMemtable(cfd, &context);
2370
    write_thread_.ExitUnbatched(&w);
2371

S
Stanislau Hlebik 已提交
2372 2373
    cfd->imm()->FlushRequested();

2374 2375 2376
    // schedule flush
    SchedulePendingFlush(cfd);
    MaybeScheduleFlushOrCompaction();
S
Stanislau Hlebik 已提交
2377
  }
S
Stanislau Hlebik 已提交
2378

L
Lei Jin 已提交
2379
  if (s.ok() && flush_options.wait) {
2380
    // Wait until the compaction completes
2381
    s = WaitForFlushMemTable(cfd);
2382 2383
  }
  return s;
J
jorlow@chromium.org 已提交
2384 2385
}

2386
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
2387 2388
  Status s;
  // Wait until the compaction completes
2389
  InstrumentedMutexLock l(&mutex_);
2390
  while (cfd->imm()->NumNotFlushed() > 0 && bg_error_.ok()) {
2391 2392 2393
    if (shutting_down_.load(std::memory_order_acquire)) {
      return Status::ShutdownInProgress();
    }
2394 2395
    bg_cv_.Wait();
  }
2396
  if (!bg_error_.ok()) {
2397 2398 2399
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
2400 2401
}

2402 2403 2404 2405
Status DBImpl::EnableAutoCompaction(
    const std::vector<ColumnFamilyHandle*>& column_family_handles) {
  Status s;
  for (auto cf_ptr : column_family_handles) {
2406 2407 2408 2409 2410 2411 2412 2413 2414 2415
    Status status =
        this->SetOptions(cf_ptr, {{"disable_auto_compactions", "false"}});
    if (status.ok()) {
      ColumnFamilyData* cfd =
          reinterpret_cast<ColumnFamilyHandleImpl*>(cf_ptr)->cfd();
      InstrumentedMutexLock guard_lock(&mutex_);
      delete this->InstallSuperVersionAndScheduleWork(
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
    } else {
      s = status;
2416 2417 2418 2419 2420 2421
    }
  }

  return s;
}

2422
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
2423
  mutex_.AssertHeld();
2424 2425 2426 2427
  if (!opened_successfully_) {
    // Compaction may introduce data race to DB open
    return;
  }
2428 2429
  if (bg_work_paused_ > 0) {
    // we paused the background work
2430
    return;
I
Igor Canadi 已提交
2431
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
2432
    // DB is being deleted; no more background compactions
2433 2434
    return;
  }
2435

2436 2437 2438 2439
  while (unscheduled_flushes_ > 0 &&
         bg_flush_scheduled_ < db_options_.max_background_flushes) {
    unscheduled_flushes_--;
    bg_flush_scheduled_++;
2440
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
2441 2442
  }

2443 2444 2445 2446 2447 2448 2449 2450 2451 2452 2453 2454
  // 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);
    }
  }

2455
  if (HasExclusiveManualCompaction()) {
2456 2457 2458 2459 2460
    // only manual compactions are allowed to run. don't schedule automatic
    // compactions
    return;
  }

2461 2462
  while (bg_compaction_scheduled_ < db_options_.max_background_compactions &&
         unscheduled_compactions_ > 0) {
2463 2464 2465
    CompactionArg* ca = new CompactionArg;
    ca->db = this;
    ca->m = nullptr;
2466 2467
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
2468 2469
    env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this,
                   &DBImpl::UnscheduleCallback);
2470 2471 2472 2473 2474 2475 2476 2477 2478 2479 2480 2481 2482 2483 2484 2485 2486 2487 2488 2489 2490 2491 2492 2493 2494 2495 2496 2497 2498 2499 2500 2501 2502 2503 2504 2505 2506 2507 2508 2509 2510 2511 2512 2513 2514 2515
  }
}

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_;
2516 2517 2518
  }
}

2519
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
2520
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
2521 2522 2523
  IOSTATS_RESET(bytes_written);
}

2524
void DBImpl::BGWorkFlush(void* db) {
2525
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
2526
  TEST_SYNC_POINT("DBImpl::BGWorkFlush");
2527
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
2528
  TEST_SYNC_POINT("DBImpl::BGWorkFlush:done");
2529 2530
}

2531 2532 2533
void DBImpl::BGWorkCompaction(void* arg) {
  CompactionArg ca = *(reinterpret_cast<CompactionArg*>(arg));
  delete reinterpret_cast<CompactionArg*>(arg);
2534
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
2535
  TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
2536 2537 2538 2539 2540 2541 2542 2543 2544 2545
  reinterpret_cast<DBImpl*>(ca.db)->BackgroundCallCompaction(ca.m);
}

void DBImpl::UnscheduleCallback(void* arg) {
  CompactionArg ca = *(reinterpret_cast<CompactionArg*>(arg));
  delete reinterpret_cast<CompactionArg*>(arg);
  if ((ca.m != nullptr) && (ca.m->compaction != nullptr)) {
    delete ca.m->compaction;
  }
  TEST_SYNC_POINT("DBImpl::UnscheduleCallback");
2546 2547
}

2548
Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
H
Haobo Xu 已提交
2549
                               LogBuffer* log_buffer) {
2550
  mutex_.AssertHeld();
2551

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

  if (!status.ok()) {
    return status;
2559 2560
  }

2561 2562 2563
  ColumnFamilyData* cfd = nullptr;
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
I
Igor Canadi 已提交
2564
    auto first_cfd = PopFirstFromFlushQueue();
2565

I
Igor Canadi 已提交
2566
    if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
2567
      // can't flush this CF, try next one
I
Igor Canadi 已提交
2568 2569
      if (first_cfd->Unref()) {
        delete first_cfd;
2570 2571
      }
      continue;
2572
    }
2573 2574

    // found a flush!
I
Igor Canadi 已提交
2575
    cfd = first_cfd;
2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587 2588
    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_);
2589
    status = FlushMemTableToOutputFile(cfd, mutable_cf_options, made_progress,
2590 2591 2592
                                       job_context, log_buffer);
    if (cfd->Unref()) {
      delete cfd;
2593
    }
J
jorlow@chromium.org 已提交
2594
  }
2595
  return status;
J
jorlow@chromium.org 已提交
2596 2597
}

2598
void DBImpl::BackgroundCallFlush() {
2599
  bool made_progress = false;
2600
  JobContext job_context(next_job_id_.fetch_add(1), true);
2601 2602
  assert(bg_flush_scheduled_);

2603
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2604
  {
2605
    InstrumentedMutexLock l(&mutex_);
2606
    num_running_flushes_++;
H
Haobo Xu 已提交
2607

I
Igor Canadi 已提交
2608 2609 2610
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2611
    Status s = BackgroundFlush(&made_progress, &job_context, &log_buffer);
2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622 2623 2624 2625 2626 2627 2628
    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 已提交
2629 2630
    }

I
Igor Canadi 已提交
2631 2632
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2633 2634 2635
    // 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 已提交
2636
    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2637
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2638
      mutex_.Unlock();
2639 2640 2641 2642 2643
      // 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 已提交
2644
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2645 2646
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2647
      }
I
Igor Canadi 已提交
2648
      job_context.Clean();
2649 2650
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2651

2652 2653
    assert(num_running_flushes_ > 0);
    num_running_flushes_--;
H
Haobo Xu 已提交
2654
    bg_flush_scheduled_--;
2655 2656
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2657
    RecordFlushIOStats();
H
Haobo Xu 已提交
2658
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2659 2660 2661 2662
    // 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.
2663
  }
J
jorlow@chromium.org 已提交
2664 2665
}

2666
void DBImpl::BackgroundCallCompaction(void* arg) {
2667
  bool made_progress = false;
2668
  ManualCompaction* m = reinterpret_cast<ManualCompaction*>(arg);
2669
  JobContext job_context(next_job_id_.fetch_add(1), true);
2670
  TEST_SYNC_POINT("BackgroundCallCompaction:0");
H
Haobo Xu 已提交
2671
  MaybeDumpStats();
2672
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2673
  {
2674
    InstrumentedMutexLock l(&mutex_);
2675
    num_running_compactions_++;
I
Igor Canadi 已提交
2676 2677 2678 2679

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2680
    assert(bg_compaction_scheduled_);
2681 2682
    Status s =
        BackgroundCompaction(&made_progress, &job_context, &log_buffer, m);
2683
    TEST_SYNC_POINT("BackgroundCallCompaction:1");
2684 2685 2686 2687 2688 2689
    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 =
2690
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2691 2692 2693 2694 2695 2696 2697 2698 2699 2700
      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();
2701
    }
H
Haobo Xu 已提交
2702

I
Igor Canadi 已提交
2703 2704
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2705 2706 2707 2708
    // 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());
2709 2710

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2711
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2712
      mutex_.Unlock();
2713 2714 2715 2716 2717
      // 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 已提交
2718
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2719 2720
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2721
      }
I
Igor Canadi 已提交
2722
      job_context.Clean();
2723 2724
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2725

2726 2727
    assert(num_running_compactions_ > 0);
    num_running_compactions_--;
2728
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2729

2730 2731
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2732 2733
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2734 2735
    if (made_progress || bg_compaction_scheduled_ == 0 ||
        HasPendingManualCompaction()) {
2736
      // signal if
2737
      // * made_progress -- need to wakeup DelayWrite
2738
      // * bg_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl
2739
      // * HasPendingManualCompaction -- need to wakeup RunManualCompaction
2740 2741 2742 2743
      // If none of this is true, there is no need to signal since nobody is
      // waiting for it
      bg_cv_.SignalAll();
    }
I
Igor Canadi 已提交
2744 2745 2746 2747
    // 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.
2748
  }
J
jorlow@chromium.org 已提交
2749 2750
}

2751 2752
Status DBImpl::BackgroundCompaction(bool* made_progress,
                                    JobContext* job_context,
2753 2754 2755
                                    LogBuffer* log_buffer, void* arg) {
  ManualCompaction* manual_compaction =
      reinterpret_cast<ManualCompaction*>(arg);
2756
  *made_progress = false;
J
jorlow@chromium.org 已提交
2757
  mutex_.AssertHeld();
2758

2759 2760 2761 2762 2763
  bool is_manual = (manual_compaction != nullptr);

  // (manual_compaction->in_progress == false);
  bool trivial_move_disallowed =
      is_manual && manual_compaction->disallow_trivial_move;
2764

2765
  CompactionJobStats compaction_job_stats;
2766 2767 2768 2769 2770 2771
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
2772
    if (is_manual) {
2773 2774 2775 2776 2777
      manual_compaction->status = status;
      manual_compaction->done = true;
      manual_compaction->in_progress = false;
      delete manual_compaction->compaction;
      manual_compaction = nullptr;
2778
    }
2779
    return status;
2780 2781
  }

2782 2783
  if (is_manual) {
    // another thread cannot pick up the same work
2784
    manual_compaction->in_progress = true;
2785 2786 2787
  }

  unique_ptr<Compaction> c;
2788 2789
  // InternalKey manual_end_storage;
  // InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2790
  if (is_manual) {
2791
    ManualCompaction* m = manual_compaction;
2792
    assert(m->in_progress);
2793
    c.reset(std::move(m->compaction));
2794
    if (!c) {
2795
      m->done = true;
2796
      m->manual_end = nullptr;
2797 2798 2799 2800 2801 2802 2803 2804 2805 2806 2807 2808 2809
      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)"),
2810
                  ((m->done || m->manual_end == nullptr)
2811
                       ? "(end)"
2812
                       : m->manual_end->DebugString().c_str()));
G
Gabor Cselle 已提交
2813
    }
2814 2815 2816 2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828
  } 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();
    }

2829 2830 2831 2832 2833 2834
    if (HaveManualCompaction(cfd)) {
      // Can't compact right now, but try again later
      TEST_SYNC_POINT("DBImpl::BackgroundCompaction()::Conflict");
      return Status::OK();
    }

2835 2836 2837 2838 2839 2840 2841 2842 2843 2844 2845 2846 2847 2848 2849 2850 2851
    // 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 已提交
2852
        // InstallSuperVersionAndScheduleWork
2853
        // 2) When MutableCFOptions changes. This case is also covered by
I
Igor Canadi 已提交
2854 2855
        // InstallSuperVersionAndScheduleWork, because this is when the new
        // options take effect.
2856 2857 2858 2859 2860 2861 2862 2863 2864 2865 2866
        // 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 已提交
2867
        }
I
Igor Canadi 已提交
2868 2869
      }
    }
J
jorlow@chromium.org 已提交
2870 2871
  }

2872
  if (!c) {
H
hans@chromium.org 已提交
2873
    // Nothing to do
2874
    LogToBuffer(log_buffer, "Compaction nothing to do");
2875
  } else if (c->deletion_compaction()) {
I
Igor Canadi 已提交
2876 2877 2878 2879
    // 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);
2880
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2881
           kCompactionStyleFIFO);
2882 2883 2884

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

I
Igor Canadi 已提交
2885
    for (const auto& f : *c->inputs(0)) {
2886
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2887
    }
2888 2889 2890
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
2891 2892
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2893 2894 2895
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
2896
    *made_progress = true;
2897
  } else if (!trivial_move_disallowed && c->IsTrivialMove()) {
2898
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2899 2900
    // Instrument for event update
    // TODO(yhchiang): add op details for showing trivial-move.
2901 2902 2903
    ThreadStatusUtil::SetColumnFamily(
        c->column_family_data(), c->column_family_data()->ioptions()->env,
        c->column_family_data()->options()->enable_thread_tracking);
2904 2905
    ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);

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

2908 2909 2910
    // Move files to next level
    int32_t moved_files = 0;
    int64_t moved_bytes = 0;
2911
    for (unsigned int l = 0; l < c->num_input_levels(); l++) {
2912
      if (c->level(l) == c->output_level()) {
2913 2914 2915 2916
        continue;
      }
      for (size_t i = 0; i < c->num_input_files(l); i++) {
        FileMetaData* f = c->input(l, i);
2917
        c->edit()->DeleteFile(c->level(l), f->fd.GetNumber());
2918 2919 2920 2921 2922 2923 2924 2925 2926 2927 2928 2929
        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();
      }
2930
    }
2931

2932
    status = versions_->LogAndApply(c->column_family_data(),
2933 2934
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2935
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2936 2937
    InstallSuperVersionAndScheduleWorkWrapper(
        c->column_family_data(), job_context, *c->mutable_cf_options());
2938

S
sdong 已提交
2939
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Islam AbdelRahman 已提交
2940
    c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(),
2941
                                                             moved_bytes);
2942 2943 2944 2945
    {
      event_logger_.LogToBuffer(log_buffer)
          << "job" << job_context->job_id << "event"
          << "trivial_move"
I
Islam AbdelRahman 已提交
2946
          << "destination_level" << c->output_level() << "files" << moved_files
2947
          << "total_files_size" << moved_bytes;
2948
    }
2949 2950
    LogToBuffer(
        log_buffer,
2951
        "[%s] Moved #%d files to level-%d %" PRIu64 " bytes %s: %s\n",
I
Islam AbdelRahman 已提交
2952 2953
        c->column_family_data()->GetName().c_str(), moved_files,
        c->output_level(), moved_bytes, status.ToString().c_str(),
2954
        c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
2955
    *made_progress = true;
2956 2957 2958

    // Clear Instrument
    ThreadStatusUtil::ResetThreadStatus();
J
jorlow@chromium.org 已提交
2959
  } else {
2960 2961 2962
    int output_level  __attribute__((unused)) = c->output_level();
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial",
                             &output_level);
2963 2964 2965 2966 2967

    SequenceNumber earliest_write_conflict_snapshot;
    std::vector<SequenceNumber> snapshot_seqs =
        snapshots_.GetAll(&earliest_write_conflict_snapshot);

I
Igor Canadi 已提交
2968
    assert(is_snapshot_supported_ || snapshots_.empty());
2969
    CompactionJob compaction_job(
I
Igor Canadi 已提交
2970 2971
        job_context->job_id, c.get(), db_options_, env_options_,
        versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(),
2972 2973
        directories_.GetDataDir(c->output_path_id()), stats_, snapshot_seqs,
        earliest_write_conflict_snapshot, table_cache_, &event_logger_,
2974 2975
        c->mutable_cf_options()->paranoid_file_checks,
        c->mutable_cf_options()->compaction_measure_io_stats, dbname_,
S
sdong 已提交
2976
        &compaction_job_stats);
I
Igor Canadi 已提交
2977
    compaction_job.Prepare();
2978

I
Igor Canadi 已提交
2979
    mutex_.Unlock();
2980
    compaction_job.Run();
2981
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun");
I
Igor Canadi 已提交
2982
    mutex_.Lock();
2983

2984
    status = compaction_job.Install(*c->mutable_cf_options(), &mutex_);
I
Igor Canadi 已提交
2985
    if (status.ok()) {
I
Igor Canadi 已提交
2986 2987
      InstallSuperVersionAndScheduleWorkWrapper(
          c->column_family_data(), job_context, *c->mutable_cf_options());
I
Igor Canadi 已提交
2988
    }
2989
    *made_progress = true;
O
Ori Bernstein 已提交
2990 2991
  }
  if (c != nullptr) {
2992 2993 2994
    NotifyOnCompactionCompleted(
        c->column_family_data(), c.get(), status,
        compaction_job_stats, job_context->job_id);
I
Igor Canadi 已提交
2995
    c->ReleaseCompactionFiles(status);
2996
    *made_progress = true;
J
jorlow@chromium.org 已提交
2997
  }
2998
  // this will unref its input_version and column_family_data
2999
  c.reset();
J
jorlow@chromium.org 已提交
3000 3001 3002

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
3003
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
3004 3005
    // Ignore compaction errors found during shutting down
  } else {
3006
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
3007
        status.ToString().c_str());
3008
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
3009 3010 3011
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
3012 3013

  if (is_manual) {
3014
    ManualCompaction* m = manual_compaction;
3015
    if (!status.ok()) {
L
Lei Jin 已提交
3016
      m->status = status;
3017 3018
      m->done = true;
    }
3019 3020 3021 3022 3023 3024 3025 3026 3027
    // 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.
3028 3029 3030 3031
    //
    // 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
3032
    if (m->manual_end == nullptr) {
3033 3034
      m->done = true;
    }
G
Gabor Cselle 已提交
3035 3036 3037
    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 已提交
3038
      // Universal and FIFO compactions should always compact the whole range
S
sdong 已提交
3039 3040 3041
      assert(m->cfd->ioptions()->compaction_style !=
                 kCompactionStyleUniversal ||
             m->cfd->ioptions()->num_levels > 1);
3042
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
3043
      m->tmp_storage = *m->manual_end;
G
Gabor Cselle 已提交
3044
      m->begin = &m->tmp_storage;
3045
      m->incomplete = true;
G
Gabor Cselle 已提交
3046
    }
3047
    m->in_progress = false; // not being processed anymore
H
hans@chromium.org 已提交
3048
  }
3049
  return status;
J
jorlow@chromium.org 已提交
3050 3051
}

3052 3053 3054 3055 3056 3057 3058 3059 3060 3061 3062 3063 3064 3065 3066 3067 3068 3069 3070 3071 3072 3073 3074
bool DBImpl::HasPendingManualCompaction() {
  return (!manual_compaction_dequeue_.empty());
}

void DBImpl::AddManualCompaction(DBImpl::ManualCompaction* m) {
  manual_compaction_dequeue_.push_back(m);
}

void DBImpl::RemoveManualCompaction(DBImpl::ManualCompaction* m) {
  // Remove from queue
  std::deque<ManualCompaction*>::iterator it =
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if (m == (*it)) {
      it = manual_compaction_dequeue_.erase(it);
      return;
    }
    it++;
  }
  assert(false);
  return;
}

3075 3076 3077
bool DBImpl::ShouldntRunManualCompaction(ManualCompaction* m) {
  if (m->exclusive) {
    return (bg_compaction_scheduled_ > 0);
3078 3079 3080 3081 3082 3083 3084 3085 3086 3087 3088 3089 3090 3091 3092 3093 3094 3095 3096 3097 3098 3099 3100 3101 3102 3103 3104 3105 3106 3107 3108 3109 3110 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
  }
  std::deque<ManualCompaction*>::iterator it =
      manual_compaction_dequeue_.begin();
  bool seen = false;
  while (it != manual_compaction_dequeue_.end()) {
    if (m == (*it)) {
      it++;
      seen = true;
      continue;
    } else if (MCOverlap(m, (*it)) && (!seen && !(*it)->in_progress)) {
      // Consider the other manual compaction *it, conflicts if:
      // overlaps with m
      // and (*it) is ahead in the queue and is not yet in progress
      return true;
    }
    it++;
  }
  return false;
}

bool DBImpl::HaveManualCompaction(ColumnFamilyData* cfd) {
  // Remove from priority queue
  std::deque<ManualCompaction*>::iterator it =
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if ((*it)->exclusive) {
      return true;
    }
    if ((cfd == (*it)->cfd) && (!((*it)->in_progress || (*it)->done))) {
      // Allow automatic compaction if manual compaction is
      // is in progress
      return true;
    }
    it++;
  }
  return false;
}

bool DBImpl::HasExclusiveManualCompaction() {
  // Remove from priority queue
  std::deque<ManualCompaction*>::iterator it =
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if ((*it)->exclusive) {
      return true;
    }
    it++;
  }
  return false;
}

bool DBImpl::MCOverlap(ManualCompaction* m, ManualCompaction* m1) {
  if ((m->exclusive) || (m1->exclusive)) {
    return true;
  }
  if (m->cfd != m1->cfd) {
    return false;
  }
  return true;
}

3139 3140
namespace {
struct IterState {
3141
  IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version)
I
Igor Canadi 已提交
3142
      : db(_db), mu(_mu), super_version(_super_version) {}
3143 3144

  DBImpl* db;
3145
  InstrumentedMutex* mu;
3146
  SuperVersion* super_version;
3147 3148 3149 3150
};

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

3152
  if (state->super_version->Unref()) {
3153 3154 3155
    // Job id == 0 means that this is not our background process, but rather
    // user thread
    JobContext job_context(0);
3156

3157 3158
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
3159
    state->db->FindObsoleteFiles(&job_context, false, true);
3160 3161 3162
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
3163 3164
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
3165
    }
I
Igor Canadi 已提交
3166
    job_context.Clean();
I
Igor Canadi 已提交
3167
  }
T
Tomislav Novak 已提交
3168

3169 3170
  delete state;
}
H
Hans Wennborg 已提交
3171
}  // namespace
3172

S
sdong 已提交
3173 3174 3175 3176 3177
InternalIterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
                                              ColumnFamilyData* cfd,
                                              SuperVersion* super_version,
                                              Arena* arena) {
  InternalIterator* internal_iter;
3178 3179 3180 3181 3182
  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 已提交
3183
      super_version->mem->NewIterator(read_options, arena));
3184
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
3185
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
3186
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
3187
  super_version->current->AddIterators(read_options, env_options_,
3188 3189
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
3190
  IterState* cleanup = new IterState(this, &mutex_, super_version);
3191
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
3192 3193 3194 3195

  return internal_iter;
}

3196 3197 3198 3199
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
3200
Status DBImpl::Get(const ReadOptions& read_options,
3201
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
3202
                   std::string* value) {
L
Lei Jin 已提交
3203
  return GetImpl(read_options, column_family, key, value);
3204 3205
}

I
Igor Canadi 已提交
3206 3207
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
3208 3209
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
3210
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
3211
//
I
Igor Canadi 已提交
3212 3213 3214
// However, if InstallSuperVersionAndScheduleWork() gets called twice with the
// same job_context, we can't reuse the SuperVersion() that got
// malloced because
I
Igor Canadi 已提交
3215 3216 3217
// 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 已提交
3218
void DBImpl::InstallSuperVersionAndScheduleWorkWrapper(
I
Igor Canadi 已提交
3219
    ColumnFamilyData* cfd, JobContext* job_context,
3220
    const MutableCFOptions& mutable_cf_options) {
3221
  mutex_.AssertHeld();
I
Igor Canadi 已提交
3222
  SuperVersion* old_superversion = InstallSuperVersionAndScheduleWork(
I
Igor Canadi 已提交
3223 3224 3225
      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 已提交
3226 3227
}

I
Igor Canadi 已提交
3228
SuperVersion* DBImpl::InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
3229
    ColumnFamilyData* cfd, SuperVersion* new_sv,
I
Igor Canadi 已提交
3230
    const MutableCFOptions& mutable_cf_options) {
L
Lei Jin 已提交
3231
  mutex_.AssertHeld();
3232 3233 3234 3235 3236 3237 3238 3239 3240

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

3244
  // Whenever we install new SuperVersion, we might need to issue new flushes or
I
Igor Canadi 已提交
3245 3246 3247 3248
  // compactions.
  SchedulePendingFlush(cfd);
  SchedulePendingCompaction(cfd);
  MaybeScheduleFlushOrCompaction();
L
Lei Jin 已提交
3249 3250 3251 3252 3253 3254 3255 3256 3257 3258

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

3264 3265 3266
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3267
  SequenceNumber snapshot;
L
Lei Jin 已提交
3268 3269 3270
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
3271 3272
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
3273
  }
3274
  // Acquire SuperVersion
3275
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
3276
  // Prepare to store a list of merge operations if merge occurs.
3277
  MergeContext merge_context;
3278

3279
  Status s;
3280
  // First look in the memtable, then in the immutable memtable (if any).
3281
  // s is both in/out. When in, s could either be OK or MergeInProgress.
3282
  // merge_operands will contain the sequence of merges in the latter case.
3283
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
3284
  PERF_TIMER_STOP(get_snapshot_time);
3285

3286
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
3287
    // Done
L
Lei Jin 已提交
3288
    RecordTick(stats_, MEMTABLE_HIT);
3289
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
3290
    // Done
L
Lei Jin 已提交
3291
    RecordTick(stats_, MEMTABLE_HIT);
3292
  } else {
3293
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
3294 3295
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
3296
    RecordTick(stats_, MEMTABLE_MISS);
3297
  }
3298

3299 3300
  {
    PERF_TIMER_GUARD(get_post_process_time);
3301

3302
    ReturnAndCleanupSuperVersion(cfd, sv);
3303

3304 3305 3306
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
3307
  return s;
J
jorlow@chromium.org 已提交
3308 3309
}

3310
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
3311
    const ReadOptions& read_options,
3312
    const std::vector<ColumnFamilyHandle*>& column_family,
3313
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
3314

L
Lei Jin 已提交
3315
  StopWatch sw(env_, stats_, DB_MULTIGET);
3316
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
3317

3318
  SequenceNumber snapshot;
3319

3320
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
3321
    ColumnFamilyData* cfd;
3322 3323 3324 3325 3326
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
3327 3328 3329 3330 3331 3332
    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});
3333 3334 3335
    }
  }

3336
  mutex_.Lock();
L
Lei Jin 已提交
3337 3338 3339
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
3340 3341 3342
  } else {
    snapshot = versions_->LastSequence();
  }
3343
  for (auto mgd_iter : multiget_cf_data) {
3344 3345
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
3346
  }
3347
  mutex_.Unlock();
3348

3349 3350
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
3351

3352
  // Note: this always resizes the values array
3353 3354 3355
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
3356 3357

  // Keep track of bytes that we read for statistics-recording later
3358
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
3359
  PERF_TIMER_STOP(get_snapshot_time);
3360 3361 3362 3363

  // 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.
3364
  // merge_operands will contain the sequence of merges in the latter case.
3365
  for (size_t i = 0; i < num_keys; ++i) {
3366
    merge_context.Clear();
3367
    Status& s = stat_list[i];
3368 3369 3370
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
3371 3372
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
3373 3374 3375
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
3376
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
3377
      // Done
3378
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
3379 3380
      // Done
    } else {
3381
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
3382 3383
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
3384 3385 3386
    }

    if (s.ok()) {
3387
      bytes_read += value->size();
3388 3389 3390 3391
    }
  }

  // Post processing (decrement reference counts and record statistics)
3392
  PERF_TIMER_GUARD(get_post_process_time);
3393 3394
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
3395
  // TODO(icanadi) do we need lock here or just around Cleanup()?
3396 3397 3398 3399 3400 3401
  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);
3402 3403
    }
  }
3404 3405 3406 3407 3408 3409 3410
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
3411
  }
3412

L
Lei Jin 已提交
3413 3414 3415
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
3416
  PERF_TIMER_STOP(get_post_process_time);
3417

3418
  return stat_list;
3419 3420
}

3421 3422 3423 3424 3425 3426 3427 3428 3429 3430 3431 3432 3433 3434 3435 3436 3437 3438 3439 3440 3441 3442 3443 3444 3445 3446 3447 3448 3449 3450 3451 3452 3453 3454 3455 3456 3457 3458 3459 3460 3461 3462 3463 3464 3465 3466 3467 3468 3469 3470 3471 3472 3473 3474 3475
#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 已提交
3476 3477
  std::unique_ptr<InternalIterator> iter(
      table_reader->NewIterator(ReadOptions()));
3478 3479 3480 3481 3482 3483 3484 3485 3486 3487 3488 3489 3490 3491 3492 3493 3494 3495 3496 3497 3498 3499 3500 3501 3502 3503 3504 3505 3506 3507

  // 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();

3508 3509 3510
  if (file_info->num_entries == 0) {
    return Status::InvalidArgument("File contain no entries");
  }
3511 3512 3513 3514 3515 3516 3517 3518 3519 3520 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
  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_);

3557 3558 3559 3560
    if (!snapshots_.empty()) {
      // Check that no snapshots are being held
      status =
          Status::NotSupported("Cannot add a file while holding snapshots");
3561 3562 3563
    }

    if (status.ok()) {
3564 3565 3566 3567 3568 3569 3570 3571 3572 3573 3574 3575 3576 3577 3578 3579 3580 3581 3582 3583 3584 3585
      // Verify that added file key range dont overlap with any keys in DB
      SuperVersion* sv = cfd->GetSuperVersion()->Ref();
      Arena arena;
      ReadOptions ro;
      ro.total_order_seek = true;
      ScopedArenaIterator iter(NewInternalIterator(ro, cfd, sv, &arena));

      InternalKey range_start(file_info->smallest_key, kMaxSequenceNumber,
                              kTypeValue);
      iter->Seek(range_start.Encode());
      status = iter->status();

      if (status.ok() && iter->Valid()) {
        ParsedInternalKey seek_result;
        if (ParseInternalKey(iter->key(), &seek_result)) {
          auto* vstorage = cfd->current()->storage_info();
          if (vstorage->InternalComparator()->user_comparator()->Compare(
                  seek_result.user_key, file_info->largest_key) <= 0) {
            status = Status::NotSupported("Cannot add overlapping range");
          }
        } else {
          status = Status::Corruption("DB have corrupted keys");
3586 3587 3588 3589 3590
        }
      }
    }

    if (status.ok()) {
3591
      // Add file to L0
3592 3593
      VersionEdit edit;
      edit.SetColumnFamily(cfd->GetID());
3594
      edit.AddFile(0, meta.fd.GetNumber(), meta.fd.GetPathId(),
3595 3596 3597 3598 3599 3600 3601 3602 3603 3604 3605 3606 3607 3608 3609 3610 3611 3612 3613 3614 3615 3616 3617 3618 3619 3620 3621 3622 3623 3624 3625 3626 3627 3628 3629 3630 3631
                   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 已提交
3632
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3633
                                  const std::string& column_family_name,
3634
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
3635
  Status s;
3636
  Status persist_options_status;
I
Igor Canadi 已提交
3637
  *handle = nullptr;
3638 3639

  s = CheckCompressionSupported(cf_options);
3640 3641 3642
  if (s.ok() && db_options_.allow_concurrent_memtable_write) {
    s = CheckConcurrentWritesSupported(cf_options);
  }
3643 3644 3645 3646
  if (!s.ok()) {
    return s;
  }

Y
Yueh-Hsuan Chiang 已提交
3647
  {
3648
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
3649

Y
Yueh-Hsuan Chiang 已提交
3650 3651 3652 3653 3654 3655 3656 3657 3658 3659 3660 3661 3662 3663
    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 已提交
3664
    {  // write thread
3665 3666
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
I
Igor Canadi 已提交
3667 3668 3669 3670
      // LogAndApply will both write the creation in MANIFEST and create
      // ColumnFamilyData object
      s = versions_->LogAndApply(
          nullptr, MutableCFOptions(opt, ImmutableCFOptions(opt)), &edit,
3671
          &mutex_, directories_.GetDbDir(), false, &cf_options);
3672 3673 3674 3675 3676 3677

      if (s.ok()) {
        // If the column family was created successfully, we then persist
        // the updated RocksDB options under the same single write thread
        persist_options_status = WriteOptionsFile();
      }
3678
      write_thread_.ExitUnbatched(&w);
I
Igor Canadi 已提交
3679
    }
Y
Yueh-Hsuan Chiang 已提交
3680 3681 3682 3683 3684
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
I
Igor Canadi 已提交
3685
      delete InstallSuperVersionAndScheduleWork(
Y
Yueh-Hsuan Chiang 已提交
3686
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
S
sdong 已提交
3687 3688 3689 3690 3691

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

Y
Yueh-Hsuan Chiang 已提交
3692 3693 3694 3695 3696 3697 3698 3699 3700
      *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());
    }
3701
  }  // InstrumentedMutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
3702 3703

  // this is outside the mutex
3704
  if (s.ok()) {
3705 3706
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
3707 3708 3709 3710 3711 3712 3713 3714 3715 3716 3717
    if (!persist_options_status.ok()) {
      if (db_options_.fail_if_options_file_error) {
        s = Status::IOError(
            "ColumnFamily has been created, but unable to persist"
            "options in CreateColumnFamily()",
            persist_options_status.ToString().c_str());
      }
      Warn(db_options_.info_log,
           "Unable to persist options in CreateColumnFamily() -- %s",
           persist_options_status.ToString().c_str());
    }
3718
  }
3719
  return s;
3720 3721
}

3722 3723 3724 3725
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
3726 3727
    return Status::InvalidArgument("Can't drop default column family");
  }
3728

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

I
Igor Canadi 已提交
3731 3732
  VersionEdit edit;
  edit.DropColumnFamily();
3733 3734
  edit.SetColumnFamily(cfd->GetID());

3735
  Status s;
3736
  Status options_persist_status;
3737
  {
3738
    InstrumentedMutexLock l(&mutex_);
3739 3740 3741 3742
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
3743
      // we drop column family from a single write thread
3744 3745
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
3746 3747
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
3748 3749 3750 3751 3752
      if (s.ok()) {
        // If the column family was dropped successfully, we then persist
        // the updated RocksDB options under the same single write thread
        options_persist_status = WriteOptionsFile();
      }
3753
      write_thread_.ExitUnbatched(&w);
3754
    }
S
sdong 已提交
3755 3756 3757 3758 3759 3760

    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()) {
3761
        if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) {
S
sdong 已提交
3762 3763 3764 3765 3766 3767
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
3768
  }
3769

3770
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
3771 3772 3773 3774
    // 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 已提交
3775
    assert(cfd->IsDropped());
L
Lei Jin 已提交
3776 3777 3778
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
3779 3780 3781
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n", cfd->GetID());

3782 3783 3784 3785 3786 3787 3788 3789 3790 3791 3792
    if (!options_persist_status.ok()) {
      if (db_options_.fail_if_options_file_error) {
        s = Status::IOError(
            "ColumnFamily has been dropped, but unable to persist "
            "options in DropColumnFamily()",
            options_persist_status.ToString().c_str());
      }
      Warn(db_options_.info_log,
           "Unable to persist options in DropColumnFamily() -- %s",
           options_persist_status.ToString().c_str());
    }
3793
  } else {
3794
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
3795
        "Dropping column family with id %u FAILED -- %s\n",
3796 3797 3798
        cfd->GetID(), s.ToString().c_str());
  }

3799
  return s;
3800 3801
}

L
Lei Jin 已提交
3802
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
3803 3804
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
3805
  if (value_found != nullptr) {
K
Kai Liu 已提交
3806 3807
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
3808
  }
L
Lei Jin 已提交
3809
  ReadOptions roptions = read_options;
3810
  roptions.read_tier = kBlockCacheTier; // read from block cache only
3811
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
3812

3813
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
3814 3815 3816
  // 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();
3817 3818
}

3819
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
3820 3821 3822
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
3823

V
Venkatesh Radhakrishnan 已提交
3824 3825 3826 3827 3828 3829 3830 3831 3832 3833 3834 3835 3836 3837 3838 3839 3840 3841
  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 已提交
3842 3843 3844 3845
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
3846 3847
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
3848 3849
    return NewDBIterator(
        env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3850 3851
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
3852 3853
        read_options.iterate_upper_bound, read_options.prefix_same_as_start,
        read_options.pin_data);
I
Igor Canadi 已提交
3854
#endif
T
Tomislav Novak 已提交
3855
  } else {
3856
    SequenceNumber latest_snapshot = versions_->LastSequence();
3857
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
3858

I
Igor Canadi 已提交
3859
    auto snapshot =
3860 3861 3862
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
3863
            : latest_snapshot;
T
Tomislav Novak 已提交
3864

3865 3866 3867 3868 3869 3870 3871 3872 3873 3874 3875 3876 3877 3878 3879 3880 3881 3882 3883 3884 3885 3886 3887 3888 3889 3890 3891 3892 3893 3894 3895 3896 3897 3898 3899 3900
    // 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         | <----+
    // |       |   +-------------------+
    // |       |                       |
    // +-------+-----------------------+
    //
3901 3902
    // ArenaWrappedDBIter inlines an arena area where all the iterators in
    // the iterator tree are allocated in the order of being accessed when
3903 3904 3905 3906 3907
    // 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(
3908 3909
        env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
3910 3911
        read_options.iterate_upper_bound, read_options.prefix_same_as_start,
        read_options.pin_data);
3912

S
sdong 已提交
3913
    InternalIterator* internal_iter =
3914
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3915 3916 3917 3918
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
3919 3920
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
3921 3922
}

3923
Status DBImpl::NewIterators(
3924
    const ReadOptions& read_options,
I
Igor Canadi 已提交
3925
    const std::vector<ColumnFamilyHandle*>& column_families,
3926
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3927 3928
  iterators->clear();
  iterators->reserve(column_families.size());
V
Venkatesh Radhakrishnan 已提交
3929 3930 3931 3932 3933 3934 3935 3936 3937 3938 3939 3940 3941 3942 3943 3944 3945 3946 3947 3948
  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 已提交
3949 3950 3951 3952
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3953 3954
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
3955 3956
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
3957 3958 3959 3960 3961
      iterators->push_back(NewDBIterator(
          env_, *cfd->ioptions(), cfd->user_comparator(), iter,
          kMaxSequenceNumber,
          sv->mutable_cf_options.max_sequential_skip_in_iterations, nullptr,
          false, read_options.pin_data));
I
Igor Canadi 已提交
3962
    }
I
Igor Canadi 已提交
3963
#endif
I
Igor Canadi 已提交
3964
  } else {
3965 3966
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
3967
    for (size_t i = 0; i < column_families.size(); ++i) {
3968 3969 3970
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
3971 3972

      auto snapshot =
3973 3974 3975
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
3976 3977
              : latest_snapshot;

3978
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3979
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
3980 3981
          sv->mutable_cf_options.max_sequential_skip_in_iterations, nullptr,
          false, read_options.pin_data);
S
sdong 已提交
3982 3983
      InternalIterator* internal_iter =
          NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3984 3985
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3986 3987 3988 3989
    }
  }

  return Status::OK();
3990 3991
}

3992 3993
const Snapshot* DBImpl::GetSnapshot() { return GetSnapshotImpl(false); }

3994
#ifndef ROCKSDB_LITE
3995 3996 3997
const Snapshot* DBImpl::GetSnapshotForWriteConflictBoundary() {
  return GetSnapshotImpl(true);
}
3998
#endif  // ROCKSDB_LITE
3999 4000

const Snapshot* DBImpl::GetSnapshotImpl(bool is_write_conflict_boundary) {
4001 4002
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error
4003
  SnapshotImpl* s = new SnapshotImpl;
4004

4005
  InstrumentedMutexLock l(&mutex_);
4006
  // returns null if the underlying memtable does not support snapshot.
4007 4008 4009 4010
  if (!is_snapshot_supported_) {
    delete s;
    return nullptr;
  }
4011 4012
  return snapshots_.New(s, versions_->LastSequence(), unix_time,
                        is_write_conflict_boundary);
J
jorlow@chromium.org 已提交
4013 4014 4015
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
4016 4017 4018 4019 4020 4021
  const SnapshotImpl* casted_s = reinterpret_cast<const SnapshotImpl*>(s);
  {
    InstrumentedMutexLock l(&mutex_);
    snapshots_.Delete(casted_s);
  }
  delete casted_s;
J
jorlow@chromium.org 已提交
4022 4023 4024
}

// Convenience methods
4025 4026
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
4027
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
4028 4029
}

4030 4031 4032
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
4033
  if (!cfh->cfd()->ioptions()->merge_operator) {
4034 4035
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
4036
    return DB::Merge(o, column_family, key, val);
4037 4038 4039
  }
}

L
Lei Jin 已提交
4040
Status DBImpl::Delete(const WriteOptions& write_options,
4041
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
4042
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
4043 4044
}

A
Andres Noetzli 已提交
4045 4046 4047 4048 4049 4050
Status DBImpl::SingleDelete(const WriteOptions& write_options,
                            ColumnFamilyHandle* column_family,
                            const Slice& key) {
  return DB::SingleDelete(write_options, column_family, key);
}

L
Lei Jin 已提交
4051
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
A
agiardullo 已提交
4052 4053 4054
  return WriteImpl(write_options, my_batch, nullptr);
}

A
agiardullo 已提交
4055
#ifndef ROCKSDB_LITE
A
agiardullo 已提交
4056 4057 4058 4059 4060
Status DBImpl::WriteWithCallback(const WriteOptions& write_options,
                                 WriteBatch* my_batch,
                                 WriteCallback* callback) {
  return WriteImpl(write_options, my_batch, callback);
}
A
agiardullo 已提交
4061
#endif  // ROCKSDB_LITE
A
agiardullo 已提交
4062 4063 4064

Status DBImpl::WriteImpl(const WriteOptions& write_options,
                         WriteBatch* my_batch, WriteCallback* callback) {
S
Stanislau Hlebik 已提交
4065 4066 4067
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
4068 4069 4070
  if (write_options.timeout_hint_us != 0) {
    return Status::InvalidArgument("timeout_hint_us is deprecated");
  }
A
agiardullo 已提交
4071 4072

  Status status;
A
agiardullo 已提交
4073 4074
  bool callback_failed = false;

A
agiardullo 已提交
4075 4076 4077 4078 4079 4080 4081 4082 4083 4084
  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;
  }

4085
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
4086
  WriteThread::Writer w;
S
Stanislau Hlebik 已提交
4087
  w.batch = my_batch;
L
Lei Jin 已提交
4088 4089
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
4090
  w.in_batch_group = false;
A
agiardullo 已提交
4091
  w.has_callback = (callback != nullptr) ? true : false;
S
Stanislau Hlebik 已提交
4092

L
Lei Jin 已提交
4093
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
4094 4095 4096
    RecordTick(stats_, WRITE_WITH_WAL);
  }

4097 4098
  StopWatch write_sw(env_, db_options_.statistics.get(), DB_WRITE);

4099
  write_thread_.JoinBatchGroup(&w);
4100 4101 4102 4103 4104 4105 4106 4107 4108 4109 4110 4111 4112 4113 4114 4115 4116 4117 4118 4119 4120 4121 4122 4123
  if (w.state == WriteThread::STATE_PARALLEL_FOLLOWER) {
    // we are a non-leader in a parallel group
    PERF_TIMER_GUARD(write_memtable_time);

    ColumnFamilyMemTablesImpl column_family_memtables(
        versions_->GetColumnFamilySet());
    WriteBatchInternal::SetSequence(w.batch, w.sequence);
    w.status = WriteBatchInternal::InsertInto(
        w.batch, &column_family_memtables, &flush_scheduler_,
        write_options.ignore_missing_column_families, 0 /*log_number*/, this,
        true /*dont_filter_deletes*/, true /*concurrent_memtable_writes*/);

    if (write_thread_.CompleteParallelWorker(&w)) {
      // we're responsible for early exit
      auto last_sequence = w.parallel_group->last_writer->sequence;
      SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
      versions_->SetLastSequence(last_sequence);
      write_thread_.EarlyExitParallelGroup(&w);
    }
    assert(w.state == WriteThread::STATE_COMPLETED);
    // STATE_COMPLETED conditional below handles exit
  }
  if (w.state == WriteThread::STATE_COMPLETED) {
    // write is complete and leader has updated sequence
4124 4125 4126 4127
    RecordTick(stats_, WRITE_DONE_BY_OTHER);
    return w.status;
  }
  // else we are the leader of the write batch group
4128
  assert(w.state == WriteThread::STATE_GROUP_LEADER);
4129

S
Stanislau Hlebik 已提交
4130 4131
  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
4132 4133 4134 4135 4136

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

S
Stanislau Hlebik 已提交
4137 4138
  RecordTick(stats_, WRITE_DONE_BY_SELF);
  default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_SELF, 1);
4139

4140 4141 4142 4143
  // 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.
4144 4145 4146
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

4147
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
4148
                                    ? 4 * max_total_in_memory_state_
4149
                                    : db_options_.max_total_wal_size;
4150 4151 4152
  if (UNLIKELY(!single_column_family_mode_ &&
               alive_log_files_.begin()->getting_flushed == false &&
               total_log_size_ > max_total_wal_size)) {
I
Igor Canadi 已提交
4153
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
4154
    alive_log_files_.begin()->getting_flushed = true;
4155
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
4156 4157 4158
        "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 已提交
4159 4160
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
4161
    for (auto cfd : *versions_->GetColumnFamilySet()) {
4162 4163 4164
      if (cfd->IsDropped()) {
        continue;
      }
I
Igor Canadi 已提交
4165
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
I
Igor Canadi 已提交
4166
        status = SwitchMemtable(cfd, &context);
I
Igor Canadi 已提交
4167 4168 4169
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
4170
        cfd->imm()->FlushRequested();
4171
        SchedulePendingFlush(cfd);
4172
      }
4173
    }
I
Igor Canadi 已提交
4174
    MaybeScheduleFlushOrCompaction();
4175 4176
  } else if (UNLIKELY(write_buffer_.ShouldFlush())) {
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
4177 4178
        "Flushing column family with largest mem table size. Write buffer is "
        "using %" PRIu64 " bytes out of a total of %" PRIu64 ".",
4179 4180 4181
        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
4182 4183 4184
    ColumnFamilyData* largest_cfd = nullptr;
    size_t largest_cfd_size = 0;

4185
    for (auto cfd : *versions_->GetColumnFamilySet()) {
4186 4187 4188
      if (cfd->IsDropped()) {
        continue;
      }
4189
      if (!cfd->mem()->IsEmpty()) {
4190 4191 4192 4193 4194 4195
        // We only consider active mem table, hoping immutable memtable is
        // already in the process of flushing.
        size_t cfd_size = cfd->mem()->ApproximateMemoryUsage();
        if (largest_cfd == nullptr || cfd_size > largest_cfd_size) {
          largest_cfd = cfd;
          largest_cfd_size = cfd_size;
4196 4197 4198
        }
      }
    }
4199 4200 4201 4202 4203 4204 4205 4206
    if (largest_cfd != nullptr) {
      status = SwitchMemtable(largest_cfd, &context);
      if (status.ok()) {
        largest_cfd->imm()->FlushRequested();
        SchedulePendingFlush(largest_cfd);
        MaybeScheduleFlushOrCompaction();
      }
    }
I
Igor Canadi 已提交
4207 4208 4209 4210 4211 4212 4213 4214 4215 4216
  }

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

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

4217 4218
  if (UNLIKELY(status.ok() && (write_controller_.IsStopped() ||
                               write_controller_.NeedsDelay()))) {
4219 4220
    PERF_TIMER_STOP(write_pre_and_post_process_time);
    PERF_TIMER_GUARD(write_delay_time);
S
sdong 已提交
4221 4222 4223 4224
    // 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.
4225
    status = DelayWrite(last_batch_group_size_);
4226
    PERF_TIMER_START(write_pre_and_post_process_time);
I
Igor Canadi 已提交
4227 4228
  }

D
dgrogan@chromium.org 已提交
4229
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
4230
  WriteThread::Writer* last_writer = &w;
A
agiardullo 已提交
4231
  autovector<WriteBatch*> write_batch_group;
4232 4233
  bool need_log_sync = !write_options.disableWAL && write_options.sync;
  bool need_log_dir_sync = need_log_sync && !log_dir_synced_;
A
agiardullo 已提交
4234

S
Stanislau Hlebik 已提交
4235
  if (status.ok()) {
4236
    if (need_log_sync) {
4237 4238 4239 4240 4241 4242 4243 4244 4245
      while (logs_.front().getting_synced) {
        log_sync_cv_.Wait();
      }
      for (auto& log : logs_) {
        assert(!log.getting_synced);
        log.getting_synced = true;
      }
    }

4246 4247 4248
    // 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
4249
    // into memtables
A
agiardullo 已提交
4250 4251 4252 4253 4254 4255 4256

    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 已提交
4257
      callback_failed = true;
A
agiardullo 已提交
4258 4259 4260 4261 4262 4263 4264
    }
  } else {
    mutex_.Unlock();
  }

  // At this point the mutex is unlocked

4265 4266 4267 4268
  bool exit_completed_early = false;
  last_batch_group_size_ = write_thread_.EnterAsBatchGroupLeader(
      &w, &last_writer, &write_batch_group);

A
agiardullo 已提交
4269
  if (status.ok()) {
4270 4271 4272 4273 4274 4275 4276 4277 4278 4279 4280 4281 4282 4283 4284 4285 4286 4287 4288 4289 4290 4291 4292 4293 4294 4295 4296 4297 4298 4299 4300
    // Rules for when we can update the memtable concurrently
    // 1. supported by memtable
    // 2. Puts are not okay if inplace_update_support
    // 3. Deletes or SingleDeletes are not okay if filtering deletes
    //    (controlled by both batch and memtable setting)
    // 4. Merges are not okay
    //
    // Rules 1..3 are enforced by checking the options
    // during startup (CheckConcurrentWritesSupported), so if
    // options.allow_concurrent_memtable_write is true then they can be
    // assumed to be true.  Rule 4 is checked for each batch.  We could
    // relax rules 2 and 3 if we could prevent write batches from referring
    // more than once to a particular key.
    bool parallel = db_options_.allow_concurrent_memtable_write &&
                    write_batch_group.size() > 1;
    int total_count = 0;
    uint64_t total_byte_size = 0;
    for (auto b : write_batch_group) {
      total_count += WriteBatchInternal::Count(b);
      total_byte_size = WriteBatchInternal::AppendedByteSize(
          total_byte_size, WriteBatchInternal::ByteSize(b));
      parallel = parallel && !b->HasMerge();
    }

    const SequenceNumber current_sequence = last_sequence + 1;
    last_sequence += total_count;

    // Record statistics
    RecordTick(stats_, NUMBER_KEYS_WRITTEN, total_count);
    RecordTick(stats_, BYTES_WRITTEN, total_byte_size);
    PERF_TIMER_STOP(write_pre_and_post_process_time);
4301

4302 4303 4304
    if (write_options.disableWAL) {
      flush_on_destroy_ = true;
    }
4305

4306 4307 4308
    uint64_t log_size = 0;
    if (!write_options.disableWAL) {
      PERF_TIMER_GUARD(write_wal_time);
4309

4310 4311 4312 4313 4314 4315 4316 4317 4318 4319 4320 4321 4322 4323 4324 4325 4326 4327 4328 4329 4330 4331 4332 4333 4334 4335 4336 4337 4338 4339 4340 4341 4342 4343 4344 4345 4346 4347 4348 4349 4350 4351 4352 4353 4354 4355
      WriteBatch* merged_batch = nullptr;
      if (write_batch_group.size() == 1) {
        merged_batch = write_batch_group[0];
      } else {
        // WAL needs all of the batches flattened into a single batch.
        // We could avoid copying here with an iov-like AddRecord
        // interface
        merged_batch = &tmp_batch_;
        for (auto b : write_batch_group) {
          WriteBatchInternal::Append(merged_batch, b);
        }
      }
      WriteBatchInternal::SetSequence(merged_batch, current_sequence);

      assert(WriteBatchInternal::Count(merged_batch) == total_count);
      assert(WriteBatchInternal::ByteSize(merged_batch) == total_byte_size);

      Slice log_entry = WriteBatchInternal::Contents(merged_batch);
      status = logs_.back().writer->AddRecord(log_entry);
      total_log_size_ += log_entry.size();
      alive_log_files_.back().AddSize(log_entry.size());
      log_empty_ = false;
      log_size = log_entry.size();
      RecordTick(stats_, WAL_FILE_BYTES, log_size);
      if (status.ok() && need_log_sync) {
        RecordTick(stats_, WAL_FILE_SYNCED);
        StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
        // 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_) {
          status = log.writer->file()->Sync(db_options_.use_fsync);
          if (!status.ok()) {
            break;
          }
        }
        if (status.ok() && need_log_dir_sync) {
          // 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();
        }
4356 4357
      }

4358 4359 4360 4361 4362 4363
      if (merged_batch == &tmp_batch_) {
        tmp_batch_.Clear();
      }
    }
    if (status.ok()) {
      PERF_TIMER_GUARD(write_memtable_time);
4364

4365 4366 4367 4368 4369 4370 4371 4372 4373 4374 4375 4376
      {
        // Update stats while we are an exclusive group leader, so we know
        // that nobody else can be writing to these particular stats.
        // We're optimistic, updating the stats before we successfully
        // commit.  That lets us release our leader status early in
        // some cases.
        auto stats = default_cf_internal_stats_;
        stats->AddDBStats(InternalStats::BYTES_WRITTEN, total_byte_size);
        stats->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN, total_count);
        if (!write_options.disableWAL) {
          if (write_options.sync) {
            stats->AddDBStats(InternalStats::WAL_FILE_SYNCED, 1);
4377
          }
4378
          stats->AddDBStats(InternalStats::WAL_FILE_BYTES, log_size);
4379
        }
4380 4381 4382 4383 4384
        uint64_t for_other = write_batch_group.size() - 1;
        if (for_other > 0) {
          stats->AddDBStats(InternalStats::WRITE_DONE_BY_OTHER, for_other);
          if (!write_options.disableWAL) {
            stats->AddDBStats(InternalStats::WRITE_WITH_WAL, for_other);
4385
          }
H
heyongqiang 已提交
4386
        }
4387
      }
Y
Yueh-Hsuan Chiang 已提交
4388

4389
      if (!parallel) {
4390
        status = WriteBatchInternal::InsertInto(
4391
            write_batch_group, current_sequence, column_family_memtables_.get(),
4392 4393 4394
            &flush_scheduler_, write_options.ignore_missing_column_families,
            0 /*log_number*/, this, false /*dont_filter_deletes*/);
      } else {
4395
        WriteThread::ParallelGroup pg;
4396 4397 4398
        pg.leader = &w;
        pg.last_writer = last_writer;
        pg.early_exit_allowed = !need_log_sync;
4399 4400
        pg.running.store(static_cast<uint32_t>(write_batch_group.size()),
                         std::memory_order_relaxed);
4401 4402 4403 4404 4405 4406 4407 4408 4409 4410 4411 4412 4413 4414 4415 4416 4417 4418
        write_thread_.LaunchParallelFollowers(&pg, current_sequence);

        ColumnFamilyMemTablesImpl column_family_memtables(
            versions_->GetColumnFamilySet());
        assert(w.sequence == current_sequence);
        WriteBatchInternal::SetSequence(w.batch, w.sequence);
        w.status = WriteBatchInternal::InsertInto(
            w.batch, &column_family_memtables, &flush_scheduler_,
            write_options.ignore_missing_column_families, 0 /*log_number*/,
            this, true /*dont_filter_deletes*/,
            true /*concurrent_memtable_writes*/);

        assert(last_writer->sequence == last_sequence);
        // CompleteParallelWorker returns true if this thread should
        // handle exit, false means somebody else did
        exit_completed_early = !write_thread_.CompleteParallelWorker(&w);
        status = w.status;
        assert(status.ok() || !exit_completed_early);
4419
      }
A
agiardullo 已提交
4420

4421 4422 4423 4424 4425 4426
      if (status.ok() && !exit_completed_early) {
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
        versions_->SetLastSequence(last_sequence);
        if (!need_log_sync) {
          write_thread_.ExitAsBatchGroupLeader(&w, last_writer, status);
          exit_completed_early = true;
4427
        }
4428
      }
4429 4430 4431 4432 4433 4434 4435 4436 4437 4438 4439

      // A non-OK status here indicates that the state implied by the
      // WAL has diverged from the in-memory state.  This could be
      // because of a corrupt write_batch (very bad), or because the
      // client specified an invalid column family and didn't specify
      // ignore_missing_column_families.
      //
      // Is setting bg_error_ enough here?  This will at least stop
      // compaction and fail any further writes.
      if (!status.ok() && bg_error_.ok()) {
        bg_error_ = status;
4440
      }
4441
    }
4442
  }
4443
  PERF_TIMER_START(write_pre_and_post_process_time);
A
agiardullo 已提交
4444

A
agiardullo 已提交
4445
  if (db_options_.paranoid_checks && !status.ok() && !callback_failed &&
4446 4447 4448 4449 4450 4451
      !status.IsBusy()) {
    mutex_.Lock();
    if (bg_error_.ok()) {
      bg_error_ = status;  // stop compaction & fail any further writes
    }
    mutex_.Unlock();
I
Igor Canadi 已提交
4452
  }
4453

4454
  if (need_log_sync) {
4455
    mutex_.Lock();
4456
    MarkLogsSynced(logfile_number_, need_log_dir_sync, status);
4457
    mutex_.Unlock();
4458 4459
  }

4460 4461
  if (!exit_completed_early) {
    write_thread_.ExitAsBatchGroupLeader(&w, last_writer, status);
4462
  }
4463

J
jorlow@chromium.org 已提交
4464 4465 4466
  return status;
}

4467 4468
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
4469
Status DBImpl::DelayWrite(uint64_t num_bytes) {
S
sdong 已提交
4470 4471 4472 4473
  uint64_t time_delayed = 0;
  bool delayed = false;
  {
    StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed);
S
sdong 已提交
4474 4475
    auto delay = write_controller_.GetDelay(env_, num_bytes);
    if (delay > 0) {
S
sdong 已提交
4476 4477
      mutex_.Unlock();
      delayed = true;
S
sdong 已提交
4478
      TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep");
4479 4480
      // hopefully we don't have to sleep more than 2 billion microseconds
      env_->SleepForMicroseconds(static_cast<int>(delay));
S
sdong 已提交
4481 4482
      mutex_.Lock();
    }
4483

4484
    while (bg_error_.ok() && write_controller_.IsStopped()) {
S
sdong 已提交
4485
      delayed = true;
4486 4487
      TEST_SYNC_POINT("DBImpl::DelayWrite:Wait");
      bg_cv_.Wait();
4488 4489
    }
  }
S
sdong 已提交
4490 4491 4492 4493 4494
  if (delayed) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_STALL_MICROS,
                                           time_delayed);
    RecordTick(stats_, STALL_MICROS, time_delayed);
  }
I
Igor Canadi 已提交
4495 4496

  return bg_error_;
4497 4498
}

I
Igor Canadi 已提交
4499 4500
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  ColumnFamilyData* cfd;
4501
  while ((cfd = flush_scheduler_.TakeNextColumnFamily()) != nullptr) {
I
Igor Canadi 已提交
4502
    auto status = SwitchMemtable(cfd, context);
I
Igor Canadi 已提交
4503 4504 4505 4506 4507
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
4508 4509
    }
  }
I
Igor Canadi 已提交
4510
  return Status::OK();
S
Stanislau Hlebik 已提交
4511 4512 4513 4514
}

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
4515
Status DBImpl::SwitchMemtable(ColumnFamilyData* cfd, WriteContext* context) {
S
Stanislau Hlebik 已提交
4516 4517 4518 4519 4520 4521 4522
  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.
4523
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
4524
  bool creating_new_log = !log_empty_;
S
Sage Weil 已提交
4525 4526 4527 4528 4529 4530
  uint64_t recycle_log_number = 0;
  if (creating_new_log && db_options_.recycle_log_file_num &&
      !log_recycle_files.empty()) {
    recycle_log_number = log_recycle_files.front();
    log_recycle_files.pop_front();
  }
S
Stanislau Hlebik 已提交
4531 4532 4533
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
4534
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
4535 4536 4537 4538
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
4539 4540
      EnvOptions opt_env_opt =
          env_->OptimizeForLogWrite(env_options_, db_options_);
S
Sage Weil 已提交
4541 4542 4543 4544 4545 4546 4547 4548
      if (recycle_log_number) {
        Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
            "reusing log %" PRIu64 " from recycle list\n", recycle_log_number);
        s = env_->ReuseWritableFile(
            LogFileName(db_options_.wal_dir, new_log_number),
            LogFileName(db_options_.wal_dir, recycle_log_number), &lfile,
            opt_env_opt);
      } else {
S
sdong 已提交
4549 4550 4551
        s = NewWritableFile(env_,
                            LogFileName(db_options_.wal_dir, new_log_number),
                            &lfile, opt_env_opt);
S
Sage Weil 已提交
4552
      }
S
Stanislau Hlebik 已提交
4553 4554 4555
      if (s.ok()) {
        // Our final size should be less than write_buffer_size
        // (compression, etc) but err on the side of caution.
D
Dmitri Smirnov 已提交
4556 4557 4558
        lfile->SetPreallocationBlockSize(
            mutable_cf_options.write_buffer_size / 10 +
            mutable_cf_options.write_buffer_size);
4559 4560
        unique_ptr<WritableFileWriter> file_writer(
            new WritableFileWriter(std::move(lfile), opt_env_opt));
S
sdong 已提交
4561 4562
        new_log = new log::Writer(std::move(file_writer), new_log_number,
                                  db_options_.recycle_log_file_num > 0);
I
Igor Canadi 已提交
4563
      }
S
Stanislau Hlebik 已提交
4564 4565 4566
    }

    if (s.ok()) {
A
agiardullo 已提交
4567 4568
      SequenceNumber seq = versions_->LastSequence();
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options, seq);
S
Stanislau Hlebik 已提交
4569 4570 4571
      new_superversion = new SuperVersion();
    }
  }
4572
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
4573 4574
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
4575 4576 4577 4578 4579 4580 4581 4582 4583 4584 4585 4586
  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;
4587
    log_dir_synced_ = false;
4588
    logs_.emplace_back(logfile_number_, new_log);
S
Stanislau Hlebik 已提交
4589
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
4590
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
4591 4592 4593 4594
      // 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 已提交
4595
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
4596
          loop_cfd->imm()->NumNotFlushed() == 0) {
I
Igor Canadi 已提交
4597
        loop_cfd->SetLogNumber(logfile_number_);
4598
      }
4599 4600
    }
  }
S
Stanislau Hlebik 已提交
4601
  cfd->mem()->SetNextLogNumber(logfile_number_);
4602
  cfd->imm()->Add(cfd->mem(), &context->memtables_to_free_);
S
Stanislau Hlebik 已提交
4603 4604
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
I
Igor Canadi 已提交
4605 4606
  context->superversions_to_free_.push_back(InstallSuperVersionAndScheduleWork(
      cfd, new_superversion, mutable_cf_options));
4607 4608 4609
  return s;
}

I
Igor Canadi 已提交
4610
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
4611 4612 4613 4614 4615
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

4616 4617
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
4618
  auto version = cfd->current();
4619 4620 4621 4622 4623 4624 4625 4626 4627 4628 4629 4630
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
4631 4632

Status DBImpl::GetPropertiesOfTablesInRange(ColumnFamilyHandle* column_family,
4633
                                            const Range* range, std::size_t n,
4634 4635 4636 4637 4638 4639 4640 4641 4642 4643 4644 4645 4646 4647 4648 4649 4650 4651 4652 4653
                                            TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

  // Increment the ref count
  mutex_.Lock();
  auto version = cfd->current();
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfTablesInRange(range, n, props);

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

  return s;
}

I
Igor Canadi 已提交
4654
#endif  // ROCKSDB_LITE
4655

I
Igor Canadi 已提交
4656 4657 4658 4659
const std::string& DBImpl::GetName() const {
  return dbname_;
}

4660 4661 4662 4663
Env* DBImpl::GetEnv() const {
  return env_;
}

4664 4665
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
4666
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
4667 4668
}

4669 4670
const DBOptions& DBImpl::GetDBOptions() const { return db_options_; }

4671
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
4672
                         const Slice& property, std::string* value) {
4673 4674
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4675 4676 4677
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

4678
  value->clear();
4679
  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
4680 4681
  if (is_int_property) {
    uint64_t int_value;
4682 4683
    bool ret_value = GetIntPropertyInternal(
        cfd, property_type, need_out_of_mutex, false, &int_value);
4684
    if (ret_value) {
4685
      *value = ToString(int_value);
4686 4687 4688
    }
    return ret_value;
  } else {
4689
    InstrumentedMutexLock l(&mutex_);
4690 4691 4692
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
4693 4694
}

4695 4696
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
4697 4698
  bool is_int_property = false;
  bool need_out_of_mutex = false;
4699 4700 4701 4702 4703
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);
  if (!is_int_property) {
    return false;
  }
4704 4705
  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  return GetIntPropertyInternal(cfd, property_type, need_out_of_mutex, false,
4706 4707 4708
                                value);
}

4709
bool DBImpl::GetIntPropertyInternal(ColumnFamilyData* cfd,
4710
                                    DBPropertyType property_type,
4711 4712
                                    bool need_out_of_mutex, bool is_locked,
                                    uint64_t* value) {
4713
  if (!need_out_of_mutex) {
4714 4715 4716 4717 4718 4719 4720
    if (is_locked) {
      mutex_.AssertHeld();
      return cfd->internal_stats()->GetIntProperty(property_type, value, this);
    } else {
      InstrumentedMutexLock l(&mutex_);
      return cfd->internal_stats()->GetIntProperty(property_type, value, this);
    }
4721
  } else {
4722 4723 4724 4725 4726 4727
    SuperVersion* sv = nullptr;
    if (!is_locked) {
      sv = GetAndRefSuperVersion(cfd);
    } else {
      sv = cfd->GetSuperVersion();
    }
4728 4729 4730 4731

    bool ret = cfd->internal_stats()->GetIntPropertyOutOfMutex(
        property_type, sv->current, value);

4732 4733 4734
    if (!is_locked) {
      ReturnAndCleanupSuperVersion(cfd, sv);
    }
4735 4736 4737 4738 4739

    return ret;
  }
}

4740 4741 4742 4743 4744 4745 4746 4747 4748 4749 4750 4751 4752 4753 4754 4755 4756 4757 4758 4759 4760 4761 4762 4763 4764 4765 4766 4767
bool DBImpl::GetAggregatedIntProperty(const Slice& property,
                                      uint64_t* aggregated_value) {
  bool need_out_of_mutex;
  bool is_int_property;
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);
  if (!is_int_property) {
    return false;
  }

  uint64_t sum = 0;
  {
    // Needs mutex to protect the list of column families.
    InstrumentedMutexLock l(&mutex_);
    uint64_t value;
    for (auto* cfd : *versions_->GetColumnFamilySet()) {
      if (GetIntPropertyInternal(cfd, property_type, need_out_of_mutex, true,
                                 &value)) {
        sum += value;
      } else {
        return false;
      }
    }
  }
  *aggregated_value = sum;
  return true;
}

4768 4769
SuperVersion* DBImpl::GetAndRefSuperVersion(ColumnFamilyData* cfd) {
  // TODO(ljin): consider using GetReferencedSuperVersion() directly
I
Igor Canadi 已提交
4770
  return cfd->GetThreadLocalSuperVersion(&mutex_);
4771 4772
}

A
agiardullo 已提交
4773 4774 4775 4776 4777 4778 4779 4780 4781 4782 4783 4784
// 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 已提交
4785 4786 4787 4788 4789 4790 4791 4792 4793 4794 4795 4796 4797 4798 4799 4800
// 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);
}

4801 4802
void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
4803
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
4804 4805 4806 4807 4808

  if (unref_sv) {
    // Release SuperVersion
    if (sv->Unref()) {
      {
4809
        InstrumentedMutexLock l(&mutex_);
4810 4811 4812 4813 4814 4815 4816
        sv->Cleanup();
      }
      delete sv;
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
    }
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
  }
4817 4818
}

A
agiardullo 已提交
4819 4820 4821 4822 4823 4824 4825 4826 4827 4828 4829 4830
// 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 已提交
4831 4832 4833 4834 4835 4836 4837 4838 4839 4840 4841 4842 4843 4844 4845 4846
// 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 已提交
4847 4848 4849 4850 4851 4852 4853 4854 4855 4856 4857 4858
// 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 已提交
4859 4860 4861 4862 4863 4864 4865 4866 4867 4868 4869 4870 4871 4872
// 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();
}

4873
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
4874 4875
                                 const Range* range, int n, uint64_t* sizes,
                                 bool include_memtable) {
J
jorlow@chromium.org 已提交
4876
  Version* v;
4877 4878
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
4879 4880
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
  v = sv->current;
J
jorlow@chromium.org 已提交
4881 4882 4883

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
4884 4885 4886
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
    sizes[i] = versions_->ApproximateSize(v, k1.Encode(), k2.Encode());
4887 4888 4889 4890
    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 已提交
4891 4892
  }

4893
  ReturnAndCleanupSuperVersion(cfd, sv);
J
jorlow@chromium.org 已提交
4894 4895
}

I
Igor Canadi 已提交
4896 4897 4898 4899 4900 4901 4902 4903 4904 4905 4906 4907 4908 4909 4910 4911
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 已提交
4912 4913 4914 4915 4916
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
4917
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
4918 4919 4920
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
4921
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
4922 4923
}

4924 4925 4926
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
4927 4928 4929
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
4930 4931
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
4932 4933 4934
    return Status::InvalidArgument("Invalid file name");
  }

4935 4936 4937 4938
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
4939 4940
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
4941
          name.c_str());
4942 4943
      return Status::NotSupported("Delete only supported for archived logs");
    }
4944
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
4945
    if (!status.ok()) {
4946 4947
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
4948
          name.c_str(), status.ToString().c_str());
4949 4950 4951 4952
    }
    return status;
  }

4953
  int level;
I
Igor Canadi 已提交
4954
  FileMetaData* metadata;
4955
  ColumnFamilyData* cfd;
4956
  VersionEdit edit;
4957
  JobContext job_context(next_job_id_.fetch_add(1), true);
D
Dhruba Borthakur 已提交
4958
  {
4959
    InstrumentedMutexLock l(&mutex_);
4960
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
4961
    if (!status.ok()) {
4962 4963
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
4964
      job_context.Clean();
D
Dhruba Borthakur 已提交
4965 4966
      return Status::InvalidArgument("File not found");
    }
4967
    assert(level < cfd->NumberLevels());
4968

D
Dhruba Borthakur 已提交
4969
    // If the file is being compacted no need to delete.
4970
    if (metadata->being_compacted) {
4971
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
4972
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
4973
      job_context.Clean();
D
Dhruba Borthakur 已提交
4974
      return Status::OK();
4975 4976
    }

D
Dhruba Borthakur 已提交
4977 4978 4979
    // 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 已提交
4980
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
4981
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
4982
      if (vstoreage->NumLevelFiles(i) != 0) {
4983
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
4984
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
4985
        job_context.Clean();
D
Dhruba Borthakur 已提交
4986 4987 4988
        return Status::InvalidArgument("File not in last level");
      }
    }
4989
    // if level == 0, it has to be the oldest file
S
sdong 已提交
4990 4991
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
4992 4993
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
4994
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
4995
      job_context.Clean();
4996 4997 4998
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
4999
    edit.DeleteFile(level, number);
5000
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
5001
                                    &edit, &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
5002
    if (status.ok()) {
I
Igor Canadi 已提交
5003 5004
      InstallSuperVersionAndScheduleWorkWrapper(
          cfd, &job_context, *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
5005
    }
I
Igor Canadi 已提交
5006 5007
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
5008

5009 5010 5011 5012 5013 5014 5015 5016 5017 5018 5019 5020 5021 5022 5023 5024
  LogFlush(db_options_.info_log);
  // remove files outside the db-lock
  if (job_context.HaveSomethingToDelete()) {
    // Call PurgeObsoleteFiles() without holding mutex.
    PurgeObsoleteFiles(job_context);
  }
  job_context.Clean();
  return status;
}

Status DBImpl::DeleteFilesInRange(ColumnFamilyHandle* column_family,
                                  const Slice* begin, const Slice* end) {
  Status status;
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  ColumnFamilyData* cfd = cfh->cfd();
  VersionEdit edit;
5025
  std::vector<FileMetaData*> deleted_files;
5026 5027 5028
  JobContext job_context(next_job_id_.fetch_add(1), true);
  {
    InstrumentedMutexLock l(&mutex_);
5029
    Version* input_version = cfd->current();
5030

5031 5032
    auto* vstorage = input_version->storage_info();
    for (int i = 1; i < cfd->NumberLevels(); i++) {
5033 5034 5035 5036 5037 5038 5039 5040 5041 5042 5043 5044 5045 5046 5047 5048 5049 5050 5051 5052 5053
      if (vstorage->LevelFiles(i).empty() ||
          !vstorage->OverlapInLevel(i, begin, end)) {
        continue;
      }
      std::vector<FileMetaData*> level_files;
      InternalKey begin_storage, end_storage, *begin_key, *end_key;
      if (begin == nullptr) {
        begin_key = nullptr;
      } else {
        begin_storage.SetMaxPossibleForUserKey(*begin);
        begin_key = &begin_storage;
      }
      if (end == nullptr) {
        end_key = nullptr;
      } else {
        end_storage.SetMinPossibleForUserKey(*end);
        end_key = &end_storage;
      }

      vstorage->GetOverlappingInputs(i, begin_key, end_key, &level_files, -1,
                                     nullptr, false);
5054 5055 5056
      FileMetaData* level_file;
      for (uint32_t j = 0; j < level_files.size(); j++) {
        level_file = level_files[j];
5057 5058 5059 5060 5061 5062 5063 5064 5065 5066 5067
        if (((begin == nullptr) ||
             (cfd->internal_comparator().user_comparator()->Compare(
                  level_file->smallest.user_key(), *begin) >= 0)) &&
            ((end == nullptr) ||
             (cfd->internal_comparator().user_comparator()->Compare(
                  level_file->largest.user_key(), *end) <= 0))) {
          if (level_file->being_compacted) {
            continue;
          }
          edit.SetColumnFamily(cfd->GetID());
          edit.DeleteFile(i, level_file->fd.GetNumber());
5068 5069
          deleted_files.push_back(level_file);
          level_file->being_compacted = true;
5070 5071 5072 5073
        }
      }
    }
    if (edit.GetDeletedFiles().empty()) {
5074
      job_context.Clean();
5075 5076
      return Status::OK();
    }
5077
    input_version->Ref();
5078 5079 5080 5081 5082 5083
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                    &edit, &mutex_, directories_.GetDbDir());
    if (status.ok()) {
      InstallSuperVersionAndScheduleWorkWrapper(
          cfd, &job_context, *cfd->GetLatestMutableCFOptions());
    }
5084 5085 5086 5087
    for (auto* deleted_file : deleted_files) {
      deleted_file->being_compacted = false;
    }
    input_version->Unref();
5088 5089 5090
    FindObsoleteFiles(&job_context, false);
  }  // lock released here

5091
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
5092
  // remove files outside the db-lock
I
Igor Canadi 已提交
5093
  if (job_context.HaveSomethingToDelete()) {
5094
    // Call PurgeObsoleteFiles() without holding mutex.
I
Igor Canadi 已提交
5095
    PurgeObsoleteFiles(job_context);
5096
  }
I
Igor Canadi 已提交
5097
  job_context.Clean();
5098 5099 5100
  return status;
}

5101
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
5102
  InstrumentedMutexLock l(&mutex_);
5103
  versions_->GetLiveFilesMetaData(metadata);
5104
}
5105 5106 5107 5108 5109 5110 5111 5112 5113 5114 5115

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 已提交
5116
#endif  // ROCKSDB_LITE
5117

I
Igor Canadi 已提交
5118 5119 5120 5121 5122 5123 5124
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
5128 5129
    uint64_t fsize = 0;
    Status s = env_->GetFileSize(file_path, &fsize);
D
dyniusz 已提交
5130 5131 5132 5133
    if (!s.ok() &&
        env_->GetFileSize(Rocks2LevelTableFileName(file_path), &fsize).ok()) {
      s = Status::OK();
    }
I
Igor Canadi 已提交
5134 5135 5136 5137
    if (!s.ok()) {
      corruption_messages +=
          "Can't access " + md.name + ": " + s.ToString() + "\n";
    } else if (fsize != md.size) {
5138
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
5139
                             ". Size recorded in manifest " +
5140 5141
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
5142 5143 5144 5145 5146 5147 5148 5149 5150
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

5151
Status DBImpl::GetDbIdentity(std::string& identity) const {
5152 5153
  std::string idfilename = IdentityFileName(dbname_);
  const EnvOptions soptions;
5154 5155 5156 5157 5158 5159 5160 5161 5162
  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)));
5163
  }
5164

5165 5166 5167 5168 5169
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
D
Dmitri Smirnov 已提交
5170
  char* buffer = reinterpret_cast<char*>(alloca(file_size));
5171
  Slice id;
5172
  s = id_file_reader->Read(static_cast<size_t>(file_size), &id, buffer);
5173 5174 5175 5176 5177 5178 5179 5180 5181 5182 5183
  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 已提交
5184 5185
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
5186
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
5187
               const Slice& key, const Slice& value) {
5188 5189 5190 5191
  // 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);
5192
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
5193 5194 5195
  return Write(opt, &batch);
}

5196 5197
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
5198
  WriteBatch batch;
5199
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
5200 5201 5202
  return Write(opt, &batch);
}

A
Andres Noetzli 已提交
5203 5204 5205 5206 5207 5208 5209
Status DB::SingleDelete(const WriteOptions& opt,
                        ColumnFamilyHandle* column_family, const Slice& key) {
  WriteBatch batch;
  batch.SingleDelete(column_family, key);
  return Write(opt, &batch);
}

5210 5211
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
5212
  WriteBatch batch;
5213
  batch.Merge(column_family, key, value);
5214 5215 5216
  return Write(opt, &batch);
}

5217
// Default implementation -- returns not supported status
L
Lei Jin 已提交
5218
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
5219
                              const std::string& column_family_name,
5220
                              ColumnFamilyHandle** handle) {
5221
  return Status::NotSupported("");
5222
}
5223
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
5224
  return Status::NotSupported("");
5225 5226
}

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

J
Jim Paton 已提交
5229
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
5230 5231 5232 5233
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
5234
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
5235
  std::vector<ColumnFamilyHandle*> handles;
5236
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
5237 5238 5239 5240 5241 5242 5243
  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;
5244 5245
}

5246 5247
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
5248
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
5249
  Status s = SanitizeOptionsByTable(db_options, column_families);
5250 5251 5252
  if (!s.ok()) {
    return s;
  }
5253

5254 5255
  for (auto& cfd : column_families) {
    s = CheckCompressionSupported(cfd.options);
5256 5257 5258
    if (s.ok() && db_options.allow_concurrent_memtable_write) {
      s = CheckConcurrentWritesSupported(cfd.options);
    }
5259 5260 5261 5262
    if (!s.ok()) {
      return s;
    }
    if (db_options.db_paths.size() > 1) {
5263 5264
      if ((cfd.options.compaction_style != kCompactionStyleUniversal) &&
          (cfd.options.compaction_style != kCompactionStyleLevel)) {
5265 5266
        return Status::NotSupported(
            "More than one DB paths are only supported in "
5267
            "universal and level compaction styles. ");
5268 5269
      }
    }
5270
  }
5271

5272 5273 5274
  if (db_options.db_paths.size() > 4) {
    return Status::NotSupported(
        "More than four DB paths are not supported yet. ");
5275 5276
  }

5277
  *dbptr = nullptr;
5278
  handles->clear();
J
jorlow@chromium.org 已提交
5279

I
Igor Canadi 已提交
5280 5281 5282 5283
  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);
5284
  }
5285

I
Igor Canadi 已提交
5286
  DBImpl* impl = new DBImpl(db_options, dbname);
5287
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
5288
  if (s.ok()) {
5289
    for (auto db_path : impl->db_options_.db_paths) {
5290
      s = impl->env_->CreateDirIfMissing(db_path.path);
5291 5292 5293 5294 5295 5296
      if (!s.ok()) {
        break;
      }
    }
  }

5297 5298 5299 5300 5301 5302
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
5303 5304 5305 5306
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
5307
  impl->mutex_.Lock();
5308 5309
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
5310
  if (s.ok()) {
5311
    uint64_t new_log_number = impl->versions_->NewFileNumber();
5312
    unique_ptr<WritableFile> lfile;
5313
    EnvOptions soptions(db_options);
5314 5315
    EnvOptions opt_env_options =
        impl->db_options_.env->OptimizeForLogWrite(soptions, impl->db_options_);
S
sdong 已提交
5316 5317 5318
    s = NewWritableFile(impl->db_options_.env,
                        LogFileName(impl->db_options_.wal_dir, new_log_number),
                        &lfile, opt_env_options);
J
jorlow@chromium.org 已提交
5319
    if (s.ok()) {
D
Dmitri Smirnov 已提交
5320
      lfile->SetPreallocationBlockSize((max_write_buffer_size / 10) + max_write_buffer_size);
5321
      impl->logfile_number_ = new_log_number;
5322 5323
      unique_ptr<WritableFileWriter> file_writer(
          new WritableFileWriter(std::move(lfile), opt_env_options));
5324 5325
      impl->logs_.emplace_back(
          new_log_number,
S
sdong 已提交
5326 5327
          new log::Writer(std::move(file_writer), new_log_number,
                          impl->db_options_.recycle_log_file_num > 0));
I
Igor Canadi 已提交
5328

5329 5330
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
5331 5332
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
5333 5334 5335
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
5336
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
5337 5338 5339 5340 5341 5342 5343 5344 5345 5346 5347 5348 5349 5350 5351 5352
        } 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 已提交
5353
        }
5354
      }
I
Igor Canadi 已提交
5355 5356
    }
    if (s.ok()) {
5357
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
I
Igor Canadi 已提交
5358
        delete impl->InstallSuperVersionAndScheduleWork(
L
Lei Jin 已提交
5359
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
5360
      }
I
Igor Canadi 已提交
5361 5362
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
5363
      impl->DeleteObsoleteFiles();
5364
      s = impl->directories_.GetDbDir()->Fsync();
J
jorlow@chromium.org 已提交
5365 5366
    }
  }
5367

I
Igor Canadi 已提交
5368 5369
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
S
sdong 已提交
5370
      if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
5371
        auto* vstorage = cfd->current()->storage_info();
5372
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
5373
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
5374
          if (num_files > 0) {
I
Igor Canadi 已提交
5375 5376
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
S
sdong 已提交
5377
                "open with FIFO compaction style.");
I
Igor Canadi 已提交
5378 5379 5380 5381
            break;
          }
        }
      }
S
sdong 已提交
5382 5383 5384
      if (!cfd->mem()->IsSnapshotSupported()) {
        impl->is_snapshot_supported_ = false;
      }
5385
      if (cfd->ioptions()->merge_operator != nullptr &&
5386 5387 5388 5389 5390
          !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 已提交
5391
      if (!s.ok()) {
5392 5393 5394 5395
        break;
      }
    }
  }
5396
  TEST_SYNC_POINT("DBImpl::Open:Opened");
5397
  Status persist_options_status;
5398
  if (s.ok()) {
5399 5400 5401 5402
    // Persist RocksDB Options before scheduling the compaction.
    // The WriteOptionsFile() will release and lock the mutex internally.
    persist_options_status = impl->WriteOptionsFile();

5403
    *dbptr = impl;
5404 5405 5406
    impl->opened_successfully_ = true;
    impl->MaybeScheduleFlushOrCompaction();
  }
5407 5408
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
5409
  if (s.ok()) {
S
sdong 已提交
5410 5411
    Log(InfoLogLevel::INFO_LEVEL, impl->db_options_.info_log, "DB pointer %p",
        impl);
5412
    LogFlush(impl->db_options_.info_log);
5413 5414 5415 5416 5417 5418 5419 5420 5421 5422 5423
    if (!persist_options_status.ok()) {
      if (db_options.fail_if_options_file_error) {
        s = Status::IOError(
            "DB::Open() failed --- Unable to persist Options file",
            persist_options_status.ToString());
      }
      Warn(impl->db_options_.info_log,
           "Unable to persist options in DB::Open() -- %s",
           persist_options_status.ToString().c_str());
    }
  }
5424
  if (!s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
5425
    for (auto* h : *handles) {
5426 5427
      delete h;
    }
5428
    handles->clear();
J
jorlow@chromium.org 已提交
5429
    delete impl;
5430
    *dbptr = nullptr;
J
jorlow@chromium.org 已提交
5431 5432 5433 5434
  }
  return s;
}

5435 5436 5437
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
5438
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
5439 5440
}

5441 5442 5443
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
5444
Status DestroyDB(const std::string& dbname, const Options& options) {
5445
  const InternalKeyComparator comparator(options.comparator);
5446
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
5447
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
5448
  std::vector<std::string> filenames;
5449

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

J
jorlow@chromium.org 已提交
5453
  FileLock* lock;
5454 5455
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
5456 5457 5458
  if (result.ok()) {
    uint64_t number;
    FileType type;
5459
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
5460
    for (size_t i = 0; i < filenames.size(); i++) {
5461
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
5462
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
5463
        Status del;
5464
        std::string path_to_delete = dbname + "/" + filenames[i];
K
Kosie van der Merwe 已提交
5465
        if (type == kMetaDatabase) {
5466 5467 5468
          del = DestroyDB(path_to_delete, options);
        } else if (type == kTableFile) {
          del = DeleteOrMoveToTrash(&options, path_to_delete);
K
Kosie van der Merwe 已提交
5469
        } else {
5470
          del = env->DeleteFile(path_to_delete);
K
Kosie van der Merwe 已提交
5471
        }
J
jorlow@chromium.org 已提交
5472 5473 5474 5475 5476
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
5477

5478 5479
    for (size_t path_id = 0; path_id < options.db_paths.size(); path_id++) {
      const auto& db_path = options.db_paths[path_id];
5480
      env->GetChildren(db_path.path, &filenames);
5481 5482 5483
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
5484 5485 5486 5487 5488 5489 5490
          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);
          }
5491 5492 5493 5494 5495 5496 5497
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

I
Igor Canadi 已提交
5498 5499 5500 5501 5502 5503 5504 5505 5506 5507 5508 5509 5510 5511 5512 5513 5514 5515
    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;
5516
    env->GetChildren(archivedir, &archiveFiles);
5517 5518
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
5519 5520
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
5521
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
5522 5523 5524 5525 5526
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
5527

5528
    // ignore case where no archival directory is present.
5529
    env->DeleteDir(archivedir);
5530

J
jorlow@chromium.org 已提交
5531
    env->UnlockFile(lock);  // Ignore error since state is already gone
5532
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
5533
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
5534
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
5535 5536 5537 5538
  }
  return result;
}

5539 5540
Status DBImpl::WriteOptionsFile() {
#ifndef ROCKSDB_LITE
5541
  mutex_.AssertHeld();
5542 5543 5544

  std::vector<std::string> cf_names;
  std::vector<ColumnFamilyOptions> cf_opts;
5545 5546 5547 5548 5549

  // This part requires mutex to protect the column family options
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (cfd->IsDropped()) {
      continue;
5550
    }
5551 5552 5553
    cf_names.push_back(cfd->GetName());
    cf_opts.push_back(BuildColumnFamilyOptions(
        *cfd->options(), *cfd->GetLatestMutableCFOptions()));
5554 5555
  }

5556 5557 5558 5559 5560 5561 5562 5563 5564 5565 5566 5567 5568
  // Unlock during expensive operations.  New writes cannot get here
  // because the single write thread ensures all new writes get queued.
  mutex_.Unlock();

  std::string file_name =
      TempOptionsFileName(GetName(), versions_->NewFileNumber());
  Status s = PersistRocksDBOptions(GetDBOptions(), cf_names, cf_opts, file_name,
                                   GetEnv());

  if (s.ok()) {
    s = RenameTempFileToOptionsFile(file_name);
  }
  mutex_.Lock();
5569 5570 5571 5572 5573 5574 5575 5576 5577 5578 5579 5580 5581 5582 5583 5584 5585 5586 5587 5588 5589 5590 5591 5592 5593 5594 5595 5596 5597 5598 5599 5600 5601 5602 5603 5604 5605 5606 5607 5608 5609 5610 5611 5612 5613 5614 5615 5616 5617 5618 5619 5620 5621 5622 5623 5624 5625 5626 5627 5628 5629 5630 5631 5632 5633 5634 5635 5636 5637 5638 5639
  return s;
#else
  return Status::OK();
#endif  // !ROCKSDB_LITE
}

#ifndef ROCKSDB_LITE
namespace {
void DeleteOptionsFilesHelper(const std::map<uint64_t, std::string>& filenames,
                              const size_t num_files_to_keep,
                              const std::shared_ptr<Logger>& info_log,
                              Env* env) {
  if (filenames.size() <= num_files_to_keep) {
    return;
  }
  for (auto iter = std::next(filenames.begin(), num_files_to_keep);
       iter != filenames.end(); ++iter) {
    if (!env->DeleteFile(iter->second).ok()) {
      Warn(info_log, "Unable to delete options file %s", iter->second.c_str());
    }
  }
}
}  // namespace
#endif  // !ROCKSDB_LITE

Status DBImpl::DeleteObsoleteOptionsFiles() {
#ifndef ROCKSDB_LITE
  std::vector<std::string> filenames;
  // use ordered map to store keep the filenames sorted from the newest
  // to the oldest.
  std::map<uint64_t, std::string> options_filenames;
  Status s;
  s = GetEnv()->GetChildren(GetName(), &filenames);
  if (!s.ok()) {
    return s;
  }
  for (auto& filename : filenames) {
    uint64_t file_number;
    FileType type;
    if (ParseFileName(filename, &file_number, &type) && type == kOptionsFile) {
      options_filenames.insert(
          {std::numeric_limits<uint64_t>::max() - file_number,
           GetName() + "/" + filename});
    }
  }

  // Keeps the latest 2 Options file
  const size_t kNumOptionsFilesKept = 2;
  DeleteOptionsFilesHelper(options_filenames, kNumOptionsFilesKept,
                           db_options_.info_log, GetEnv());
  return Status::OK();
#else
  return Status::OK();
#endif  // !ROCKSDB_LITE
}

Status DBImpl::RenameTempFileToOptionsFile(const std::string& file_name) {
#ifndef ROCKSDB_LITE
  Status s;
  std::string options_file_name =
      OptionsFileName(GetName(), versions_->NewFileNumber());
  // Retry if the file name happen to conflict with an existing one.
  s = GetEnv()->RenameFile(file_name, options_file_name);

  DeleteObsoleteOptionsFiles();
  return s;
#else
  return Status::OK();
#endif  // !ROCKSDB_LITE
}

Y
Yueh-Hsuan Chiang 已提交
5640
#if ROCKSDB_USING_THREAD_STATUS
5641

Y
Yueh-Hsuan Chiang 已提交
5642 5643
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
5644
  if (db_options_.enable_thread_tracking) {
5645 5646
    ThreadStatusUtil::NewColumnFamilyInfo(this, cfd, cfd->GetName(),
                                          cfd->ioptions()->env);
5647
  }
Y
Yueh-Hsuan Chiang 已提交
5648 5649 5650 5651
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
5652
  if (db_options_.enable_thread_tracking) {
5653
    ThreadStatusUtil::EraseColumnFamilyInfo(cfd);
5654
  }
Y
Yueh-Hsuan Chiang 已提交
5655 5656 5657
}

void DBImpl::EraseThreadStatusDbInfo() const {
5658
  if (db_options_.enable_thread_tracking) {
5659
    ThreadStatusUtil::EraseDatabaseInfo(this);
5660
  }
Y
Yueh-Hsuan Chiang 已提交
5661 5662 5663 5664 5665 5666 5667 5668 5669 5670 5671 5672 5673 5674 5675
}

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

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

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

5676 5677
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
5678
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
5679
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
5680 5681
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
5682 5683 5684 5685
  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 已提交
5686
#endif
5687 5688
}

A
agiardullo 已提交
5689 5690 5691 5692 5693 5694 5695 5696 5697 5698 5699 5700 5701 5702 5703 5704 5705
#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
5706 5707 5708
Status DBImpl::GetLatestSequenceForKey(SuperVersion* sv, const Slice& key,
                                       bool cache_only, SequenceNumber* seq,
                                       bool* found_record_for_key) {
A
agiardullo 已提交
5709 5710 5711 5712 5713 5714 5715
  Status s;
  MergeContext merge_context;

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

  *seq = kMaxSequenceNumber;
5716 5717
  *found_record_for_key = false;

A
agiardullo 已提交
5718
  // Check if there is a record for this key in the latest memtable
R
Reid Horuff 已提交
5719
  sv->mem->Get(lkey, nullptr, &s, &merge_context, seq);
A
agiardullo 已提交
5720 5721 5722 5723 5724 5725 5726 5727 5728 5729 5730 5731

  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
5732
    *found_record_for_key = true;
A
agiardullo 已提交
5733 5734 5735 5736
    return Status::OK();
  }

  // Check if there is a record for this key in the immutable memtables
R
Reid Horuff 已提交
5737
  sv->imm->Get(lkey, nullptr, &s, &merge_context, seq);
A
agiardullo 已提交
5738 5739 5740 5741 5742 5743 5744 5745 5746 5747 5748 5749

  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
5750
    *found_record_for_key = true;
A
agiardullo 已提交
5751 5752 5753 5754
    return Status::OK();
  }

  // Check if there is a record for this key in the immutable memtables
R
Reid Horuff 已提交
5755
  sv->imm->GetFromHistory(lkey, nullptr, &s, &merge_context, seq);
A
agiardullo 已提交
5756 5757 5758 5759 5760 5761 5762 5763 5764 5765

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

5766 5767 5768 5769 5770 5771 5772 5773 5774 5775 5776 5777
  if (*seq != kMaxSequenceNumber) {
    // Found a sequence number, no need to check SST files
    *found_record_for_key = true;
    return Status::OK();
  }

  // TODO(agiardullo): possible optimization: consider checking cached
  // SST files if cache_only=true?
  if (!cache_only) {
    // Check tables
    ReadOptions read_options;

R
Reid Horuff 已提交
5778
    sv->current->Get(read_options, lkey, nullptr, &s, &merge_context,
5779 5780 5781 5782 5783 5784 5785 5786 5787 5788 5789 5790
                     nullptr /* value_found */, found_record_for_key, seq);

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

      return s;
    }
  }

A
agiardullo 已提交
5791 5792 5793 5794
  return Status::OK();
}
#endif  // ROCKSDB_LITE

5795
}  // namespace rocksdb