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

#include "db/db_impl.h"

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

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

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

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

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

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

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

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

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

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

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

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

I
Igor Canadi 已提交
134
  if (result.rate_limiter.get() != nullptr) {
135 136 137 138 139
    if (result.bytes_per_sync == 0) {
      result.bytes_per_sync = 1024 * 1024;
    }
  }

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

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

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

155 156
namespace {

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

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

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

  if (can_compress) {
187
    return ioptions.compression;
188 189 190 191
  } else {
    return kNoCompression;
  }
}
I
Igor Canadi 已提交
192 193

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

204
}  // namespace
205

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

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

255
  versions_.reset(new VersionSet(dbname_, &db_options_, env_options_,
256 257
                                 table_cache_.get(), &write_buffer_,
                                 &write_controller_));
I
Igor Canadi 已提交
258 259
  column_family_memtables_.reset(new ColumnFamilyMemTablesImpl(
      versions_->GetColumnFamilySet(), &flush_scheduler_));
260

I
Igor Canadi 已提交
261
  DumpRocksDBBuildVersion(db_options_.info_log.get());
262 263
  DumpDBFileSummary(db_options_, dbname_);
  db_options_.Dump(db_options_.info_log.get());
I
Igor Canadi 已提交
264
  DumpCompressionInfo(db_options_.info_log.get());
265

266
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
267 268
}

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

J
jorlow@chromium.org 已提交
283
DBImpl::~DBImpl() {
Y
Yueh-Hsuan Chiang 已提交
284
  EraseThreadStatusDbInfo();
285
  mutex_.Lock();
286

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

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

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

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

358 359 360 361
  for (auto l : logs_to_free_) {
    delete l;
  }

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

370
  LogFlush(db_options_.info_log);
J
jorlow@chromium.org 已提交
371 372 373
}

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

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

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

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

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

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

  const uint64_t now_micros = env_->NowMicros();

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

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

473
    PrintStatistics();
474 475 476
  }
}

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

490
  // if deletion is disabled, do nothing
491
  if (disable_delete_obsolete_files_ > 0) {
492 493 494
    return;
  }

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

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

I
Igor Canadi 已提交
520
  // get obsolete files
I
Igor Canadi 已提交
521 522
  versions_->GetObsoleteFiles(&job_context->sst_delete_files,
                              job_context->min_pending_output);
I
Igor Canadi 已提交
523

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

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

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

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

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

I
Igor Canadi 已提交
588 589 590 591
  // 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 已提交
592 593
    return;
  }
594

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

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

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

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

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

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

    if (keep) {
      continue;
    }

684
    std::string fname;
K
kailiu 已提交
685 686
    if (type == kTableFile) {
      // evict from cache
687
      TableCache::Evict(table_cache_.get(), number);
688
      fname = TableFileName(db_options_.db_paths, number, path_id);
689
      event_logger_.Log() << "job" << state.job_id << "event"
I
Igor Canadi 已提交
690 691
                          << "table_file_deletion"
                          << "file_number" << number;
692
    } else {
693 694
      fname = ((type == kLogFile) ?
          db_options_.wal_dir : dbname_) + "/" + to_delete;
K
kailiu 已提交
695
    }
696

I
Igor Canadi 已提交
697 698
#ifdef ROCKSDB_LITE
    Status s = env_->DeleteFile(fname);
699
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
700
        "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", state.job_id,
I
Igor Canadi 已提交
701 702 703 704 705
        fname.c_str(), type, number, s.ToString().c_str());
#else   // not ROCKSDB_LITE
    if (type == kLogFile && (db_options_.WAL_ttl_seconds > 0 ||
                             db_options_.WAL_size_limit_MB > 0)) {
      wal_manager_.ArchiveWALFile(fname, number);
K
kailiu 已提交
706 707
    } else {
      Status s = env_->DeleteFile(fname);
708
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
709
          "[JOB %d] Delete %s type=%d #%" PRIu64 " -- %s\n", state.job_id,
710
          fname.c_str(), type, number, s.ToString().c_str());
J
jorlow@chromium.org 已提交
711
    }
I
Igor Canadi 已提交
712
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
713
  }
H
heyongqiang 已提交
714

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

void DBImpl::DeleteObsoleteFiles() {
  mutex_.AssertHeld();
743
  JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
744 745 746
  FindObsoleteFiles(&job_context, true);
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
747
  }
I
Igor Canadi 已提交
748
  job_context.Clean();
749 750
}

751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766
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 已提交
767

768 769 770 771 772 773 774 775 776
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_);
777 778 779
    if (!s.ok()) {
      return s;
    }
780
  }
781

782 783 784 785 786 787 788 789
  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);
790 791 792
      if (!s.ok()) {
        return s;
      }
793
      data_dirs_.emplace_back(path_directory.release());
794
    }
795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813
  }
  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();
814

815 816 817 818 819
  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);
820 821 822 823
    if (!s.ok()) {
      return s;
    }

824
    s = env_->LockFile(LockFileName(dbname_), &db_lock_);
825 826 827
    if (!s.ok()) {
      return s;
    }
J
jorlow@chromium.org 已提交
828

829
    if (!env_->FileExists(CurrentFileName(dbname_))) {
830
      if (db_options_.create_if_missing) {
831
        s = NewDB();
832
        is_new_db = true;
833 834 835 836 837 838
        if (!s.ok()) {
          return s;
        }
      } else {
        return Status::InvalidArgument(
            dbname_, "does not exist (create_if_missing is false)");
J
jorlow@chromium.org 已提交
839 840
      }
    } else {
841
      if (db_options_.error_if_exists) {
842 843 844
        return Status::InvalidArgument(
            dbname_, "exists (error_if_exists is true)");
      }
J
jorlow@chromium.org 已提交
845
    }
M
Mayank Agarwal 已提交
846 847 848 849 850 851 852
    // Check for the IDENTITY file and create it if not there
    if (!env_->FileExists(IdentityFileName(dbname_))) {
      s = SetIdentityFile(env_, dbname_);
      if (!s.ok()) {
        return s;
      }
    }
J
jorlow@chromium.org 已提交
853 854
  }

855
  Status s = versions_->Recover(column_families, read_only);
856
  if (db_options_.paranoid_checks && s.ok()) {
I
Igor Canadi 已提交
857 858
    s = CheckConsistency();
  }
J
jorlow@chromium.org 已提交
859 860
  if (s.ok()) {
    SequenceNumber max_sequence(0);
861 862
    default_cf_handle_ = new ColumnFamilyHandleImpl(
        versions_->GetColumnFamilySet()->GetDefault(), this, &mutex_);
863
    default_cf_internal_stats_ = default_cf_handle_->cfd()->internal_stats();
864 865
    single_column_family_mode_ =
        versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1;
866 867 868 869 870

    // 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).
    //
871
    // Note that prev_log_number() is no longer used, but we pay
872
    // attention to it in case we are recovering a database
873
    // produced by an older version of rocksdb.
874
    const uint64_t min_log = versions_->MinLogNumber();
875
    const uint64_t prev_log = versions_->prev_log_number();
876
    std::vector<std::string> filenames;
877
    s = env_->GetChildren(db_options_.wal_dir, &filenames);
878 879
    if (!s.ok()) {
      return s;
880
    }
K
kailiu 已提交
881

882 883
    std::vector<uint64_t> logs;
    for (size_t i = 0; i < filenames.size(); i++) {
K
kailiu 已提交
884 885
      uint64_t number;
      FileType type;
886 887 888 889 890 891 892 893 894
      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);
        }
895
      }
J
jorlow@chromium.org 已提交
896
    }
897

H
heyongqiang 已提交
898 899 900 901 902 903
    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 已提交
904 905 906 907 908 909 910
    if (!logs.empty()) {
      // Recover in the order in which the logs were generated
      std::sort(logs.begin(), logs.end());
      s = RecoverLogFiles(logs, &max_sequence, read_only);
      if (!s.ok()) {
        // Clear memtables if recovery failed
        for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
911
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
S
Stanislau Hlebik 已提交
912 913
        }
      }
914
    }
L
Lei Jin 已提交
915
    SetTickerCount(stats_, SEQUENCE_NUMBER, versions_->LastSequence());
J
jorlow@chromium.org 已提交
916 917
  }

L
Lei Jin 已提交
918 919
  // Initial value
  max_total_in_memory_state_ = 0;
I
Igor Canadi 已提交
920
  for (auto cfd : *versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
921 922 923
    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 已提交
924 925
  }

J
jorlow@chromium.org 已提交
926 927 928
  return s;
}

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

  mutex_.AssertHeld();
S
Stanislau Hlebik 已提交
947
  Status status;
948
  std::unordered_map<int, VersionEdit> version_edits;
949
  // no need to refcount because iteration is under mutex
950 951
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    VersionEdit edit;
952 953
    edit.SetColumnFamily(cfd->GetID());
    version_edits.insert({cfd->GetID(), edit});
954
  }
955 956 957 958 959 960 961 962 963 964 965 966
  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 已提交
967

S
Stanislau Hlebik 已提交
968 969 970 971
  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.
972
    versions_->MarkFileNumberUsedDuringRecovery(log_number);
S
Stanislau Hlebik 已提交
973 974 975 976
    // Open the log file
    std::string fname = LogFileName(db_options_.wal_dir, log_number);
    unique_ptr<SequentialFile> file;
    status = env_->NewSequentialFile(fname, &file, env_options_);
J
jorlow@chromium.org 已提交
977
    if (!status.ok()) {
S
Stanislau Hlebik 已提交
978 979 980 981 982 983 984 985
      MaybeIgnoreError(&status);
      if (!status.ok()) {
        return status;
      } else {
        // Fail with one log file, but that's ok.
        // Try next one.
        continue;
      }
J
jorlow@chromium.org 已提交
986 987
    }

S
Stanislau Hlebik 已提交
988 989 990 991 992
    // Create the log reader.
    LogReporter reporter;
    reporter.env = env_;
    reporter.info_log = db_options_.info_log.get();
    reporter.fname = fname.c_str();
993
    reporter.status = (db_options_.paranoid_checks) ? &status : nullptr;
S
Stanislau Hlebik 已提交
994 995 996 997 998 999
    // We intentially make log::Reader do checksumming even if
    // paranoid_checks==false so that corruptions cause entire commits
    // to be skipped instead of propagating bad information (like overly
    // large sequence numbers).
    log::Reader reader(std::move(file), &reporter, true /*checksum*/,
                       0 /*initial_offset*/);
1000 1001
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "Recovering log #%" PRIu64 "", log_number);
S
Stanislau Hlebik 已提交
1002 1003 1004 1005 1006

    // Read all the records and add to a memtable
    std::string scratch;
    Slice record;
    WriteBatch batch;
I
Igor Canadi 已提交
1007
    while (reader.ReadRecord(&record, &scratch) && status.ok()) {
S
Stanislau Hlebik 已提交
1008 1009 1010 1011 1012 1013 1014 1015 1016 1017 1018 1019 1020 1021 1022 1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033
      if (record.size() < 12) {
        reporter.Corruption(record.size(),
                            Status::Corruption("log record too small"));
        continue;
      }
      WriteBatchInternal::SetContents(&batch, record);

      // If column family was not found, it might mean that the WAL write
      // batch references to the column family that was dropped after the
      // insert. We don't want to fail the whole write batch in that case --
      // we just ignore the update.
      // That's why we set ignore missing column families to true
      status = WriteBatchInternal::InsertInto(
          &batch, column_family_memtables_.get(), true, log_number);

      MaybeIgnoreError(&status);
      if (!status.ok()) {
        return status;
      }
      const SequenceNumber last_seq = WriteBatchInternal::Sequence(&batch) +
                                      WriteBatchInternal::Count(&batch) - 1;
      if (last_seq > *max_sequence) {
        *max_sequence = last_seq;
      }

      if (!read_only) {
I
Igor Canadi 已提交
1034 1035 1036 1037 1038 1039 1040 1041 1042 1043 1044 1045
        // we can do this because this is called before client has access to the
        // DB and there is only a single thread operating on DB
        ColumnFamilyData* cfd;

        while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
          cfd->Unref();
          // If this asserts, it means that InsertInto failed in
          // filtering updates to already-flushed column families
          assert(cfd->GetLogNumber() <= log_number);
          auto iter = version_edits.find(cfd->GetID());
          assert(iter != version_edits.end());
          VersionEdit* edit = &iter->second;
1046
          status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
I
Igor Canadi 已提交
1047 1048 1049 1050
          if (!status.ok()) {
            // Reflect errors immediately so that conditions like full
            // file-systems cause the DB::Open() to fail.
            return status;
1051
          }
L
Lei Jin 已提交
1052
          cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
1053
        }
J
jorlow@chromium.org 已提交
1054 1055 1056
      }
    }

1057 1058 1059 1060
    if (!status.ok()) {
      return status;
    }

I
Igor Canadi 已提交
1061
    flush_scheduler_.Clear();
S
Stanislau Hlebik 已提交
1062 1063 1064
    if (versions_->LastSequence() < *max_sequence) {
      versions_->SetLastSequence(*max_sequence);
    }
1065 1066
  }

1067
  if (!read_only) {
1068 1069
    // 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 已提交
1070
    auto max_log_number = log_numbers.back();
1071
    for (auto cfd : *versions_->GetColumnFamilySet()) {
1072
      auto iter = version_edits.find(cfd->GetID());
1073 1074 1075
      assert(iter != version_edits.end());
      VersionEdit* edit = &iter->second;

S
Stanislau Hlebik 已提交
1076
      if (cfd->GetLogNumber() > max_log_number) {
1077
        // Column family cfd has already flushed the data
S
Stanislau Hlebik 已提交
1078
        // from all logs. Memtable has to be empty because
1079
        // we filter the updates based on log_number
1080
        // (in WriteBatch::InsertInto)
1081 1082 1083 1084 1085 1086 1087
        assert(cfd->mem()->GetFirstSequenceNumber() == 0);
        assert(edit->NumEntries() == 0);
        continue;
      }

      // flush the final memtable (if non-empty)
      if (cfd->mem()->GetFirstSequenceNumber() != 0) {
1088
        status = WriteLevel0TableForRecovery(job_id, cfd, cfd->mem(), edit);
S
Stanislau Hlebik 已提交
1089 1090 1091 1092
        if (!status.ok()) {
          // Recovery failed
          break;
        }
L
Lei Jin 已提交
1093
        cfd->CreateNewMemtable(*cfd->GetLatestMutableCFOptions());
1094
      }
J
jorlow@chromium.org 已提交
1095

1096
      // write MANIFEST with update
S
Stanislau Hlebik 已提交
1097
      // writing log_number in the manifest means that any log file
1098 1099
      // with number strongly less than (log_number + 1) is already
      // recovered and should be ignored on next reincarnation.
S
Stanislau Hlebik 已提交
1100 1101 1102
      // 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);
1103 1104 1105
      // 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 已提交
1106
      // log number
1107
      versions_->MarkFileNumberUsedDuringRecovery(max_log_number + 1);
1108 1109
      status = versions_->LogAndApply(
          cfd, *cfd->GetLatestMutableCFOptions(), edit, &mutex_);
1110
      if (!status.ok()) {
S
Stanislau Hlebik 已提交
1111 1112
        // Recovery failed
        break;
1113 1114
      }
    }
1115
  }
I
Igor Canadi 已提交
1116

1117 1118 1119
  event_logger_.Log() << "job" << job_id << "event"
                      << "recovery_finished";

J
jorlow@chromium.org 已提交
1120 1121 1122
  return status;
}

1123 1124
Status DBImpl::WriteLevel0TableForRecovery(int job_id, ColumnFamilyData* cfd,
                                           MemTable* mem, VersionEdit* edit) {
J
jorlow@chromium.org 已提交
1125
  mutex_.AssertHeld();
1126
  const uint64_t start_micros = env_->NowMicros();
J
jorlow@chromium.org 已提交
1127
  FileMetaData meta;
1128
  meta.fd = FileDescriptor(versions_->NewFileNumber(), 0, 0);
I
Igor Canadi 已提交
1129 1130
  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();
1131 1132
  ReadOptions ro;
  ro.total_order_seek = true;
1133
  Arena arena;
1134
  Status s;
1135
  TableProperties table_properties;
1136
  {
1137 1138 1139 1140
    ScopedArenaIterator iter(mem->NewIterator(ro, &arena));
    const SequenceNumber newest_snapshot = snapshots_.GetNewest();
    const SequenceNumber earliest_seqno_in_memtable =
        mem->GetFirstSequenceNumber();
1141 1142 1143
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] [WriteLevel0TableForRecovery]"
        " Level-0 table #%" PRIu64 ": started",
1144
        cfd->GetName().c_str(), meta.fd.GetNumber());
1145

1146 1147
    bool paranoid_file_checks =
        cfd->GetLatestMutableCFOptions()->paranoid_file_checks;
1148 1149 1150 1151
    {
      mutex_.Unlock();
      s = BuildTable(
          dbname_, env_, *cfd->ioptions(), env_options_, cfd->table_cache(),
1152 1153
          iter.get(), &meta, cfd->internal_comparator(),
          cfd->int_tbl_prop_collector_factories(), newest_snapshot,
1154
          earliest_seqno_in_memtable, GetCompressionFlush(*cfd->ioptions()),
1155 1156
          cfd->ioptions()->compression_opts, paranoid_file_checks, Env::IO_HIGH,
          &table_properties);
1157
      LogFlush(db_options_.info_log);
1158 1159
      mutex_.Lock();
    }
1160 1161 1162 1163 1164 1165
  }
  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());
1166

1167 1168 1169 1170 1171
  // output to event logger
  if (s.ok()) {
    EventLoggerHelpers::LogTableFileCreation(
        &event_logger_, job_id, meta.fd.GetNumber(), meta.fd.GetFileSize(),
        table_properties);
1172
  }
1173

I
Igor Canadi 已提交
1174
  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);
1175 1176 1177 1178

  // Note that if file_size is zero, the file has been deleted and
  // should not be added to the manifest.
  int level = 0;
1179
  if (s.ok() && meta.fd.GetFileSize() > 0) {
1180 1181 1182
    edit->AddFile(level, meta.fd.GetNumber(), meta.fd.GetPathId(),
                  meta.fd.GetFileSize(), meta.smallest, meta.largest,
                  meta.smallest_seqno, meta.largest_seqno);
1183 1184
  }

L
Lei Jin 已提交
1185
  InternalStats::CompactionStats stats(1);
1186
  stats.micros = env_->NowMicros() - start_micros;
1187
  stats.bytes_written = meta.fd.GetFileSize();
M
Mark Callaghan 已提交
1188
  stats.files_out_levelnp1 = 1;
1189
  cfd->internal_stats()->AddCompactionStats(level, stats);
1190 1191
  cfd->internal_stats()->AddCFStats(
      InternalStats::BYTES_FLUSHED, meta.fd.GetFileSize());
L
Lei Jin 已提交
1192
  RecordTick(stats_, COMPACT_WRITE_BYTES, meta.fd.GetFileSize());
J
jorlow@chromium.org 已提交
1193 1194 1195
  return s;
}

1196 1197
Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
I
Igor Canadi 已提交
1198
    bool* madeProgress, JobContext* job_context, LogBuffer* log_buffer) {
1199
  mutex_.AssertHeld();
1200
  assert(cfd->imm()->size() != 0);
1201
  assert(cfd->imm()->IsFlushPending());
1202

I
Igor Canadi 已提交
1203 1204
  FlushJob flush_job(dbname_, cfd, db_options_, mutable_cf_options,
                     env_options_, versions_.get(), &mutex_, &shutting_down_,
I
Igor Canadi 已提交
1205
                     snapshots_.GetNewest(), job_context, log_buffer,
1206
                     directories_.GetDbDir(), directories_.GetDataDir(0U),
I
Igor Canadi 已提交
1207 1208
                     GetCompressionFlush(*cfd->ioptions()), stats_,
                     &event_logger_);
1209

1210 1211
  uint64_t file_number;
  Status s = flush_job.Run(&file_number);
J
jorlow@chromium.org 已提交
1212 1213

  if (s.ok()) {
I
Igor Canadi 已提交
1214
    InstallSuperVersionBackground(cfd, job_context, mutable_cf_options);
1215 1216 1217
    if (madeProgress) {
      *madeProgress = 1;
    }
S
sdong 已提交
1218
    VersionStorageInfo::LevelSummaryStorage tmp;
I
Igor Canadi 已提交
1219
    LogToBuffer(log_buffer, "[%s] Level summary: %s\n", cfd->GetName().c_str(),
S
sdong 已提交
1220
                cfd->current()->storage_info()->LevelSummary(&tmp));
1221

1222
    if (disable_delete_obsolete_files_ == 0) {
I
Igor Canadi 已提交
1223
      // add to deletion state
1224
      while (alive_log_files_.size() &&
I
Igor Canadi 已提交
1225 1226
             alive_log_files_.begin()->number < versions_->MinLogNumber()) {
        const auto& earliest = *alive_log_files_.begin();
I
Igor Canadi 已提交
1227
        job_context->log_delete_files.push_back(earliest.number);
I
Igor Canadi 已提交
1228
        total_log_size_ -= earliest.size;
1229 1230
        alive_log_files_.pop_front();
      }
1231
    }
J
jorlow@chromium.org 已提交
1232
  }
1233

1234
  if (!s.ok() && !s.IsShutdownInProgress() && db_options_.paranoid_checks &&
1235 1236 1237 1238 1239
      bg_error_.ok()) {
    // if a bad error happened (not ShutdownInProgress) and paranoid_checks is
    // true, mark DB read-only
    bg_error_ = s;
  }
1240
  RecordFlushIOStats();
1241 1242 1243
#ifndef ROCKSDB_LITE
  if (s.ok()) {
    // may temporarily unlock and lock the mutex.
1244
    NotifyOnFlushCompleted(cfd, file_number, mutable_cf_options);
1245 1246
  }
#endif  // ROCKSDB_LITE
J
jorlow@chromium.org 已提交
1247 1248 1249
  return s;
}

1250
void DBImpl::NotifyOnFlushCompleted(
1251 1252
    ColumnFamilyData* cfd, uint64_t file_number,
    const MutableCFOptions& mutable_cf_options) {
1253
#ifndef ROCKSDB_LITE
1254 1255 1256
  if (cfd->ioptions()->listeners.size() == 0U) {
    return;
  }
1257 1258 1259 1260 1261 1262
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  bool triggered_flush_slowdown =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1263
       mutable_cf_options.level0_slowdown_writes_trigger);
1264 1265
  bool triggered_flush_stop =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
1266
       mutable_cf_options.level0_stop_writes_trigger);
1267 1268 1269 1270 1271 1272 1273 1274 1275 1276 1277 1278 1279
  notifying_events_++;
  // release lock while notifying events
  mutex_.Unlock();
  // TODO(yhchiang): make db_paths dynamic.
  cfd->NotifyOnFlushCompleted(
        this, MakeTableFileName(db_options_.db_paths[0].path, file_number),
        triggered_flush_slowdown,
        triggered_flush_stop);
  mutex_.Lock();
  notifying_events_--;
  assert(notifying_events_ >= 0);
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
1280
#endif  // ROCKSDB_LITE
1281 1282
}

1283
Status DBImpl::CompactRange(ColumnFamilyHandle* column_family,
1284
                            const Slice* begin, const Slice* end,
1285 1286
                            bool reduce_level, int target_level,
                            uint32_t target_path_id) {
1287
  if (target_path_id >= db_options_.db_paths.size()) {
1288 1289 1290
    return Status::InvalidArgument("Invalid target path ID");
  }

1291 1292
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
I
Igor Canadi 已提交
1293 1294

  Status s = FlushMemTable(cfd, FlushOptions());
L
Lei Jin 已提交
1295
  if (!s.ok()) {
1296
    LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1297 1298 1299
    return s;
  }

I
Igor Canadi 已提交
1300
  int max_level_with_files = 0;
G
Gabor Cselle 已提交
1301
  {
1302
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
1303
    Version* base = cfd->current();
1304 1305
    for (int level = 1; level < base->storage_info()->num_non_empty_levels();
         level++) {
S
sdong 已提交
1306
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
G
Gabor Cselle 已提交
1307 1308 1309 1310
        max_level_with_files = level;
      }
    }
  }
1311 1312 1313 1314 1315

  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
1316 1317
                            cfd->NumberLevels() - 1, target_path_id, begin,
                            end);
1318 1319 1320 1321 1322 1323 1324 1325 1326 1327 1328
  } else {
    for (int level = 0; level <= max_level_with_files; level++) {
      // in case the compaction is unversal or if we're compacting the
      // bottom-most level, the output level will be the same as input one.
      // level 0 can never be the bottommost level (i.e. if all files are in
      // level 0, we will compact to level 1)
      if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
          cfd->ioptions()->compaction_style == kCompactionStyleFIFO ||
          (level == max_level_with_files && level > 0)) {
        s = RunManualCompaction(cfd, level, level, target_path_id, begin, end);
      } else {
1329 1330 1331 1332 1333 1334 1335
        int output_level = level + 1;
        if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
            cfd->ioptions()->level_compaction_dynamic_level_bytes &&
            level == 0) {
          output_level = ColumnFamilyData::kCompactToBaseLevel;
        }
        s = RunManualCompaction(cfd, level, output_level, target_path_id, begin,
1336 1337 1338 1339 1340
                                end);
      }
      if (!s.ok()) {
        break;
      }
1341 1342
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
      TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
1343
    }
G
Gabor Cselle 已提交
1344
  }
1345 1346 1347 1348
  if (!s.ok()) {
    LogFlush(db_options_.info_log);
    return s;
  }
1349 1350

  if (reduce_level) {
I
Igor Canadi 已提交
1351
    s = ReFitLevel(cfd, max_level_with_files, target_level);
1352
  }
1353
  LogFlush(db_options_.info_log);
L
Lei Jin 已提交
1354

1355
  {
1356
    InstrumentedMutexLock l(&mutex_);
1357 1358 1359 1360 1361
    // an automatic compaction that has been scheduled might have been
    // preempted by the manual compactions. Need to schedule it back.
    MaybeScheduleFlushOrCompaction();
  }

L
Lei Jin 已提交
1362
  return s;
1363 1364
}

1365 1366 1367 1368 1369
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 已提交
1370 1371 1372 1373
#ifdef ROCKSDB_LITE
    // not supported in lite version
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
1374 1375 1376 1377 1378 1379
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);
1380 1381 1382 1383 1384 1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399 1400 1401 1402 1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420

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

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

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

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

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

1421
  return s;
I
Igor Canadi 已提交
1422
#endif  // ROCKSDB_LITE
1423 1424
}

I
Igor Canadi 已提交
1425
#ifndef ROCKSDB_LITE
1426 1427 1428
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
1429 1430
    const int output_level, int output_path_id, JobContext* job_context,
    LogBuffer* log_buffer) {
1431 1432 1433 1434 1435 1436 1437 1438 1439 1440 1441 1442 1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453 1454 1455 1456 1457 1458 1459 1460 1461 1462
  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;
  }

1463
  std::vector<CompactionInputFiles> input_files;
1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483
  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(
1484 1485
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
1486 1487 1488 1489 1490 1491
  assert(c);
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
  assert(!c->IsDeletionCompaction());

  auto yield_callback = [&]() {
1492 1493 1494
    return CallFlushDuringCompaction(
        c->column_family_data(), *c->mutable_cf_options(),
        job_context, log_buffer);
1495
  };
I
Igor Canadi 已提交
1496
  assert(is_snapshot_supported_ || snapshots_.empty());
1497
  CompactionJob compaction_job(
I
Igor Canadi 已提交
1498 1499 1500 1501 1502
      job_context->job_id, c.get(), db_options_, env_options_, versions_.get(),
      &shutting_down_, log_buffer, directories_.GetDbDir(),
      directories_.GetDataDir(c->GetOutputPathId()), stats_,
      snapshots_.GetAll(), table_cache_, std::move(yield_callback),
      &event_logger_, c->mutable_cf_options()->paranoid_file_checks);
1503 1504 1505 1506 1507
  compaction_job.Prepare();

  mutex_.Unlock();
  Status status = compaction_job.Run();
  mutex_.Lock();
I
Igor Canadi 已提交
1508
  compaction_job.Install(&status, *c->mutable_cf_options(), &mutex_);
1509
  if (status.ok()) {
1510
    InstallSuperVersionBackground(c->column_family_data(), job_context,
1511
                                  *c->mutable_cf_options());
1512 1513 1514 1515 1516 1517 1518 1519 1520
  }
  c->ReleaseCompactionFiles(s);
  c.reset();

  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
1521 1522
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] [JOB %d] Compaction error: %s",
1523
        c->column_family_data()->GetName().c_str(), job_context->job_id,
1524 1525 1526 1527 1528 1529 1530 1531 1532 1533
        status.ToString().c_str());
    if (db_options_.paranoid_checks && bg_error_.ok()) {
      bg_error_ = status;
    }
  }

  bg_compaction_scheduled_--;

  return status;
}
I
Igor Canadi 已提交
1534
#endif  // ROCKSDB_LITE
1535

O
Ori Bernstein 已提交
1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557
void DBImpl::NotifyOnCompactionCompleted(
    ColumnFamilyData* cfd, Compaction *c, const Status &st) {
#ifndef ROCKSDB_LITE
  if (cfd->ioptions()->listeners.size() == 0U) {
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  notifying_events_++;
  // release lock while notifying events
  mutex_.Unlock();
  cfd->NotifyOnCompactionCompleted(this, c, st);
  mutex_.Lock();
  notifying_events_--;
  assert(notifying_events_ >= 0);
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
#endif  // ROCKSDB_LITE
}

1558
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
1559
    const std::unordered_map<std::string, std::string>& options_map) {
I
Igor Canadi 已提交
1560 1561 1562
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
L
Lei Jin 已提交
1563 1564
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
1565 1566
    Log(InfoLogLevel::WARN_LEVEL,
        db_options_.info_log, "SetOptions() on column family [%s], empty input",
L
Lei Jin 已提交
1567
        cfd->GetName().c_str());
1568
    return Status::InvalidArgument("empty input");
L
Lei Jin 已提交
1569 1570 1571
  }

  MutableCFOptions new_options;
1572
  Status s;
L
Lei Jin 已提交
1573
  {
1574
    InstrumentedMutexLock l(&mutex_);
1575 1576
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
L
Lei Jin 已提交
1577 1578 1579 1580
      new_options = *cfd->GetLatestMutableCFOptions();
    }
  }

1581 1582
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "SetOptions() on column family [%s], inputs:",
L
Lei Jin 已提交
1583 1584
      cfd->GetName().c_str());
  for (const auto& o : options_map) {
1585 1586
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "%s: %s\n", o.first.c_str(), o.second.c_str());
L
Lei Jin 已提交
1587
  }
1588
  if (s.ok()) {
1589 1590
    Log(InfoLogLevel::INFO_LEVEL,
        db_options_.info_log, "[%s] SetOptions succeeded",
L
Lei Jin 已提交
1591 1592 1593
        cfd->GetName().c_str());
    new_options.Dump(db_options_.info_log.get());
  } else {
1594 1595
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
        "[%s] SetOptions failed", cfd->GetName().c_str());
L
Lei Jin 已提交
1596
  }
1597
  return s;
I
Igor Canadi 已提交
1598
#endif  // ROCKSDB_LITE
1599 1600
}

1601
// return the same level if it cannot be moved
1602 1603
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
1604
  mutex_.AssertHeld();
S
sdong 已提交
1605
  const auto* vstorage = cfd->current()->storage_info();
1606
  int minimum_level = level;
1607
  for (int i = level - 1; i > 0; --i) {
1608
    // stop if level i is not empty
S
sdong 已提交
1609
    if (vstorage->NumLevelFiles(i) > 0) break;
1610
    // stop if level i is too small (cannot fit the level files)
1611
    if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) {
1612 1613
      break;
    }
1614 1615 1616 1617 1618 1619

    minimum_level = i;
  }
  return minimum_level;
}

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

I
Igor Canadi 已提交
1623
  SuperVersion* superversion_to_free = nullptr;
1624
  SuperVersion* new_superversion = new SuperVersion();
I
Igor Canadi 已提交
1625 1626

  mutex_.Lock();
1627 1628 1629

  // only allow one thread refitting
  if (refitting_level_) {
I
Igor Canadi 已提交
1630
    mutex_.Unlock();
1631 1632
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[ReFitLevel] another thread is refitting");
I
Igor Canadi 已提交
1633
    delete new_superversion;
L
Lei Jin 已提交
1634
    return Status::NotSupported("another thread is refitting");
1635 1636 1637 1638 1639
  }
  refitting_level_ = true;

  // wait for all background threads to stop
  bg_work_gate_closed_ = true;
1640
  while (bg_compaction_scheduled_ > 0 || bg_flush_scheduled_) {
1641 1642
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "[RefitLevel] waiting for background threads to stop: %d %d",
1643
        bg_compaction_scheduled_, bg_flush_scheduled_);
1644 1645 1646
    bg_cv_.Wait();
  }

1647 1648
  const MutableCFOptions mutable_cf_options =
    *cfd->GetLatestMutableCFOptions();
1649
  // move to a smaller level
1650 1651
  int to_level = target_level;
  if (target_level < 0) {
1652
    to_level = FindMinimumEmptyLevelFitting(cfd, mutable_cf_options, level);
1653
  }
1654 1655 1656

  assert(to_level <= level);

L
Lei Jin 已提交
1657
  Status status;
1658
  if (to_level < level) {
1659 1660
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Before refitting:\n%s",
1661
        cfd->GetName().c_str(), cfd->current()->DebugString().data());
1662

1663
    VersionEdit edit;
I
Igor Canadi 已提交
1664
    edit.SetColumnFamily(cfd->GetID());
S
sdong 已提交
1665
    for (const auto& f : cfd->current()->storage_info()->LevelFiles(level)) {
1666
      edit.DeleteFile(level, f->fd.GetNumber());
1667 1668 1669
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
                   f->smallest_seqno, f->largest_seqno);
1670
    }
1671 1672
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] Apply version edit:\n%s",
I
Igor Canadi 已提交
1673
        cfd->GetName().c_str(), edit.DebugString().data());
1674

1675 1676
    status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                    directories_.GetDbDir());
L
Lei Jin 已提交
1677 1678
    superversion_to_free = InstallSuperVersion(
        cfd, new_superversion, mutable_cf_options);
I
Igor Canadi 已提交
1679
    new_superversion = nullptr;
1680

1681 1682
    Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
        "[%s] LogAndApply: %s\n", cfd->GetName().c_str(),
I
Igor Canadi 已提交
1683
        status.ToString().data());
1684 1685

    if (status.ok()) {
1686 1687
      Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
          "[%s] After refitting:\n%s",
I
Igor Canadi 已提交
1688
          cfd->GetName().c_str(), cfd->current()->DebugString().data());
1689 1690 1691 1692 1693
    }
  }

  refitting_level_ = false;
  bg_work_gate_closed_ = false;
I
Igor Canadi 已提交
1694 1695 1696 1697

  mutex_.Unlock();
  delete superversion_to_free;
  delete new_superversion;
L
Lei Jin 已提交
1698
  return status;
G
Gabor Cselle 已提交
1699 1700
}

1701 1702 1703
int DBImpl::NumberLevels(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  return cfh->cfd()->NumberLevels();
1704 1705
}

1706 1707
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1708
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
1709 1710
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.max_mem_compaction_level;
1711 1712
}

1713 1714
int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1715
  InstrumentedMutexLock l(&mutex_);
L
Lei Jin 已提交
1716 1717
  return cfh->cfd()->GetSuperVersion()->
      mutable_cf_options.level0_stop_writes_trigger;
1718 1719
}

L
Lei Jin 已提交
1720
Status DBImpl::Flush(const FlushOptions& flush_options,
1721 1722
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
L
Lei Jin 已提交
1723
  return FlushMemTable(cfh->cfd(), flush_options);
H
heyongqiang 已提交
1724 1725
}

1726
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
1727 1728 1729
  return versions_->LastSequence();
}

I
Igor Canadi 已提交
1730
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
1731 1732
                                   int output_level, uint32_t output_path_id,
                                   const Slice* begin, const Slice* end) {
1733 1734
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);
1735

G
Gabor Cselle 已提交
1736 1737
  InternalKey begin_storage, end_storage;

H
hans@chromium.org 已提交
1738
  ManualCompaction manual;
I
Igor Canadi 已提交
1739
  manual.cfd = cfd;
1740 1741
  manual.input_level = input_level;
  manual.output_level = output_level;
1742
  manual.output_path_id = output_path_id;
G
Gabor Cselle 已提交
1743
  manual.done = false;
1744
  manual.in_progress = false;
1745 1746 1747
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
1748 1749
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1750
    manual.begin = nullptr;
G
Gabor Cselle 已提交
1751
  } else {
1752
    begin_storage.SetMaxPossibleForUserKey(*begin);
G
Gabor Cselle 已提交
1753 1754
    manual.begin = &begin_storage;
  }
1755
  if (end == nullptr ||
1756 1757
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
1758
    manual.end = nullptr;
G
Gabor Cselle 已提交
1759
  } else {
1760
    end_storage.SetMinPossibleForUserKey(*end);
G
Gabor Cselle 已提交
1761 1762 1763
    manual.end = &end_storage;
  }

1764
  InstrumentedMutexLock l(&mutex_);
1765

1766 1767 1768 1769 1770 1771
  // When a manual compaction arrives, temporarily disable scheduling of
  // non-manual compactions and wait until the number of scheduled compaction
  // jobs drops to zero. This is needed to ensure that this manual compaction
  // can compact any range of keys/files.
  //
  // bg_manual_only_ is non-zero when at least one thread is inside
1772
  // RunManualCompaction(), i.e. during that time no other compaction will
1773 1774 1775
  // get scheduled (see MaybeScheduleFlushOrCompaction).
  //
  // Note that the following loop doesn't stop more that one thread calling
1776
  // RunManualCompaction() from getting to the second while loop below.
1777 1778 1779 1780 1781
  // However, only one of them will actually schedule compaction, while
  // others will wait on a condition variable until it completes.

  ++bg_manual_only_;
  while (bg_compaction_scheduled_ > 0) {
1782
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
1783 1784 1785
        "[%s] Manual compaction waiting for all other scheduled background "
        "compactions to finish",
        cfd->GetName().c_str());
1786 1787
    bg_cv_.Wait();
  }
1788

1789 1790
  Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
      "[%s] Manual compaction starting",
I
Igor Canadi 已提交
1791
      cfd->GetName().c_str());
1792

1793 1794 1795 1796
  // 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) {
1797 1798 1799
    assert(bg_manual_only_ > 0);
    if (manual_compaction_ != nullptr) {
      // Running either this or some other manual compaction
G
Gabor Cselle 已提交
1800
      bg_cv_.Wait();
1801 1802
    } else {
      manual_compaction_ = &manual;
1803
      bg_compaction_scheduled_++;
1804
      env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
G
Gabor Cselle 已提交
1805
    }
H
hans@chromium.org 已提交
1806
  }
1807

1808 1809 1810
  assert(!manual.in_progress);
  assert(bg_manual_only_ > 0);
  --bg_manual_only_;
L
Lei Jin 已提交
1811
  return manual.status;
J
jorlow@chromium.org 已提交
1812 1813
}

1814
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
L
Lei Jin 已提交
1815
                             const FlushOptions& flush_options) {
S
Stanislau Hlebik 已提交
1816 1817 1818
  Status s;
  {
    WriteContext context;
1819
    InstrumentedMutexLock guard_lock(&mutex_);
1820 1821 1822 1823 1824 1825

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

I
Igor Canadi 已提交
1826 1827
    WriteThread::Writer w(&mutex_);
    s = write_thread_.EnterWriteThread(&w, 0);
S
Stanislau Hlebik 已提交
1828 1829 1830 1831 1832
    assert(s.ok() && !w.done);  // No timeout and nobody should do our job

    // SetNewMemtableAndNewLogFile() will release and reacquire mutex
    // during execution
    s = SetNewMemtableAndNewLogFile(cfd, &context);
1833 1834
    write_thread_.ExitWriteThread(&w, &w, s);

S
Stanislau Hlebik 已提交
1835 1836
    cfd->imm()->FlushRequested();

1837 1838 1839
    // schedule flush
    SchedulePendingFlush(cfd);
    MaybeScheduleFlushOrCompaction();
S
Stanislau Hlebik 已提交
1840
  }
S
Stanislau Hlebik 已提交
1841

L
Lei Jin 已提交
1842
  if (s.ok() && flush_options.wait) {
1843
    // Wait until the compaction completes
1844
    s = WaitForFlushMemTable(cfd);
1845 1846
  }
  return s;
J
jorlow@chromium.org 已提交
1847 1848
}

1849
Status DBImpl::WaitForFlushMemTable(ColumnFamilyData* cfd) {
1850 1851
  Status s;
  // Wait until the compaction completes
1852
  InstrumentedMutexLock l(&mutex_);
1853
  while (cfd->imm()->size() > 0 && bg_error_.ok()) {
1854 1855
    bg_cv_.Wait();
  }
1856
  if (!bg_error_.ok()) {
1857 1858 1859
    s = bg_error_;
  }
  return s;
H
heyongqiang 已提交
1860 1861
}

1862
void DBImpl::MaybeScheduleFlushOrCompaction() {
J
jorlow@chromium.org 已提交
1863
  mutex_.AssertHeld();
1864
  if (bg_work_gate_closed_) {
1865 1866
    // gate closed for background work
    return;
I
Igor Canadi 已提交
1867
  } else if (shutting_down_.load(std::memory_order_acquire)) {
J
jorlow@chromium.org 已提交
1868
    // DB is being deleted; no more background compactions
1869 1870
    return;
  }
1871

1872 1873 1874 1875
  while (unscheduled_flushes_ > 0 &&
         bg_flush_scheduled_ < db_options_.max_background_flushes) {
    unscheduled_flushes_--;
    bg_flush_scheduled_++;
1876
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
1877 1878
  }

1879 1880 1881 1882 1883 1884
  if (bg_manual_only_) {
    // only manual compactions are allowed to run. don't schedule automatic
    // compactions
    return;
  }

1885 1886 1887 1888 1889 1890 1891 1892 1893 1894
  if (db_options_.max_background_flushes == 0 &&
      bg_compaction_scheduled_ < db_options_.max_background_compactions &&
      unscheduled_flushes_ > 0) {
    // special case where flush is executed by compaction thread
    // (if max_background_flushes == 0).
    // Compaction thread will execute all the flushes
    unscheduled_flushes_ = 0;
    if (unscheduled_compactions_ > 0) {
      // bg compaction will execute one compaction
      unscheduled_compactions_--;
1895
    }
1896
    bg_compaction_scheduled_++;
1897
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
1898 1899 1900 1901 1902 1903
  }

  while (bg_compaction_scheduled_ < db_options_.max_background_compactions &&
         unscheduled_compactions_ > 0) {
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
1904
    env_->Schedule(&DBImpl::BGWorkCompaction, this, Env::Priority::LOW, this);
1905 1906 1907 1908 1909 1910 1911 1912 1913 1914 1915 1916 1917 1918 1919 1920 1921 1922 1923 1924 1925 1926 1927 1928 1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941 1942 1943 1944 1945 1946 1947 1948 1949 1950
  }
}

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_;
1951 1952 1953
  }
}

1954
void DBImpl::RecordFlushIOStats() {
I
Igor Canadi 已提交
1955
  RecordTick(stats_, FLUSH_WRITE_BYTES, IOSTATS(bytes_written));
1956 1957 1958
  IOSTATS_RESET(bytes_written);
}

1959
void DBImpl::BGWorkFlush(void* db) {
1960
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
1961 1962 1963 1964
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
}

void DBImpl::BGWorkCompaction(void* db) {
1965
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
1966
  TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
1967 1968 1969
  reinterpret_cast<DBImpl*>(db)->BackgroundCallCompaction();
}

I
Igor Canadi 已提交
1970
Status DBImpl::BackgroundFlush(bool* madeProgress, JobContext* job_context,
H
Haobo Xu 已提交
1971
                               LogBuffer* log_buffer) {
1972
  mutex_.AssertHeld();
1973

1974 1975 1976 1977 1978 1979 1980
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
    return status;
1981 1982
  }

1983 1984 1985
  ColumnFamilyData* cfd = nullptr;
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
I
Igor Canadi 已提交
1986
    auto first_cfd = PopFirstFromFlushQueue();
1987

I
Igor Canadi 已提交
1988
    if (first_cfd->IsDropped() || !first_cfd->imm()->IsFlushPending()) {
1989
      // can't flush this CF, try next one
I
Igor Canadi 已提交
1990 1991
      if (first_cfd->Unref()) {
        delete first_cfd;
1992 1993
      }
      continue;
1994
    }
1995 1996

    // found a flush!
I
Igor Canadi 已提交
1997
    cfd = first_cfd;
1998 1999 2000 2001 2002 2003 2004 2005 2006 2007 2008 2009 2010 2011 2012 2013 2014
    break;
  }

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

2020
void DBImpl::BackgroundCallFlush() {
2021
  bool madeProgress = false;
2022
  JobContext job_context(next_job_id_.fetch_add(1), true);
2023 2024
  assert(bg_flush_scheduled_);

2025
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
H
Haobo Xu 已提交
2026
  {
2027
    InstrumentedMutexLock l(&mutex_);
H
Haobo Xu 已提交
2028

I
Igor Canadi 已提交
2029 2030 2031
    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2032
    Status s = BackgroundFlush(&madeProgress, &job_context, &log_buffer);
2033 2034 2035 2036 2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049
    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 已提交
2050 2051
    }

I
Igor Canadi 已提交
2052 2053
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2054 2055 2056 2057
    // We're just cleaning up for DB::Write()
    job_context.logs_to_free = logs_to_free_;
    logs_to_free_.clear();

2058 2059 2060
    // 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 已提交
2061
    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2062
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2063
      mutex_.Unlock();
2064 2065 2066 2067 2068
      // 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 已提交
2069
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2070 2071
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2072
      }
I
Igor Canadi 已提交
2073
      job_context.Clean();
2074 2075
      mutex_.Lock();
    }
I
Igor Canadi 已提交
2076

H
Haobo Xu 已提交
2077
    bg_flush_scheduled_--;
2078 2079
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
2080
    RecordFlushIOStats();
H
Haobo Xu 已提交
2081
    bg_cv_.SignalAll();
I
Igor Canadi 已提交
2082 2083 2084 2085
    // 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.
2086
  }
J
jorlow@chromium.org 已提交
2087 2088
}

2089
void DBImpl::BackgroundCallCompaction() {
2090
  bool madeProgress = false;
2091
  JobContext job_context(next_job_id_.fetch_add(1), true);
H
Haobo Xu 已提交
2092 2093

  MaybeDumpStats();
2094
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
2095
  {
2096
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
2097 2098 2099 2100

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2101
    assert(bg_compaction_scheduled_);
2102
    Status s = BackgroundCompaction(&madeProgress, &job_context, &log_buffer);
2103 2104 2105 2106 2107 2108
    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 =
2109
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
2110 2111 2112 2113 2114 2115 2116 2117 2118 2119
      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();
2120
    }
H
Haobo Xu 已提交
2121

I
Igor Canadi 已提交
2122 2123
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

2124 2125 2126 2127
    // We're just cleaning up for DB::Write()
    job_context.logs_to_free = logs_to_free_;
    logs_to_free_.clear();

2128 2129 2130 2131
    // 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());
2132 2133

    // delete unnecessary files if any, this is done outside the mutex
I
Igor Canadi 已提交
2134
    if (job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
2135
      mutex_.Unlock();
2136 2137 2138 2139 2140
      // 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 已提交
2141
      log_buffer.FlushBufferToLog();
I
Igor Canadi 已提交
2142 2143
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2144
      }
I
Igor Canadi 已提交
2145
      job_context.Clean();
2146 2147
      mutex_.Lock();
    }
D
Dhruba Borthakur 已提交
2148

2149
    bg_compaction_scheduled_--;
J
jorlow@chromium.org 已提交
2150

2151 2152
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();

2153 2154
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2155 2156
    if (madeProgress || bg_compaction_scheduled_ == 0 || bg_manual_only_ > 0) {
      // signal if
I
Igor Canadi 已提交
2157
      // * madeProgress -- need to wakeup DelayWrite
2158 2159 2160 2161 2162 2163
      // * bg_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl
      // * bg_manual_only_ > 0 -- need to wakeup RunManualCompaction
      // If none of this is true, there is no need to signal since nobody is
      // waiting for it
      bg_cv_.SignalAll();
    }
I
Igor Canadi 已提交
2164 2165 2166 2167
    // 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.
2168
  }
J
jorlow@chromium.org 已提交
2169 2170
}

I
Igor Canadi 已提交
2171
Status DBImpl::BackgroundCompaction(bool* madeProgress, JobContext* job_context,
2172
                                    LogBuffer* log_buffer) {
2173
  *madeProgress = false;
J
jorlow@chromium.org 已提交
2174
  mutex_.AssertHeld();
2175

2176 2177
  bool is_manual = (manual_compaction_ != nullptr) &&
                   (manual_compaction_->in_progress == false);
2178

2179 2180 2181 2182 2183 2184
  Status status = bg_error_;
  if (status.ok() && shutting_down_.load(std::memory_order_acquire)) {
    status = Status::ShutdownInProgress();
  }

  if (!status.ok()) {
2185
    if (is_manual) {
2186
      manual_compaction_->status = status;
2187 2188 2189 2190
      manual_compaction_->done = true;
      manual_compaction_->in_progress = false;
      manual_compaction_ = nullptr;
    }
2191
    return status;
2192 2193
  }

2194 2195 2196
  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction_->in_progress = true;
2197 2198 2199 2200
  } else if (manual_compaction_ != nullptr) {
    // there should be no automatic compactions running when manual compaction
    // is running
    return Status::OK();
2201 2202
  }

2203 2204 2205 2206 2207 2208 2209 2210 2211 2212 2213 2214 2215 2216
  // If there are no flush threads, then compaction thread needs to execute the
  // flushes
  if (db_options_.max_background_flushes == 0) {
    // BackgroundFlush() will only execute a single flush. We keep calling it as
    // long as there's more flushes to be done
    while (!flush_queue_.empty()) {
      LogToBuffer(
          log_buffer,
          "BackgroundCompaction calling BackgroundFlush. flush slots available "
          "%d, compaction slots available %d",
          db_options_.max_background_flushes - bg_flush_scheduled_,
          db_options_.max_background_compactions - bg_compaction_scheduled_);
      auto flush_status =
          BackgroundFlush(madeProgress, job_context, log_buffer);
2217 2218 2219 2220
      // the second condition will be false when a column family is dropped. we
      // don't want to fail compaction because of that (because it might be a
      // different column family)
      if (!flush_status.ok() && !flush_status.IsShutdownInProgress()) {
2221 2222 2223 2224 2225 2226 2227
        if (is_manual) {
          manual_compaction_->status = flush_status;
          manual_compaction_->done = true;
          manual_compaction_->in_progress = false;
          manual_compaction_ = nullptr;
        }
        return flush_status;
2228 2229 2230 2231 2232
      }
    }
  }

  unique_ptr<Compaction> c;
2233 2234
  InternalKey manual_end_storage;
  InternalKey* manual_end = &manual_end_storage;
H
hans@chromium.org 已提交
2235
  if (is_manual) {
G
Gabor Cselle 已提交
2236
    ManualCompaction* m = manual_compaction_;
2237
    assert(m->in_progress);
2238 2239 2240
    c.reset(m->cfd->CompactRange(
          *m->cfd->GetLatestMutableCFOptions(), m->input_level, m->output_level,
          m->output_path_id, m->begin, m->end, &manual_end));
2241
    if (!c) {
2242
      m->done = true;
2243 2244 2245 2246 2247 2248 2249 2250 2251 2252 2253 2254 2255 2256 2257 2258
      LogToBuffer(log_buffer,
                  "[%s] Manual compaction from level-%d from %s .. "
                  "%s; nothing to do\n",
                  m->cfd->GetName().c_str(), m->input_level,
                  (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                  (m->end ? m->end->DebugString().c_str() : "(end)"));
    } else {
      LogToBuffer(log_buffer,
                  "[%s] Manual compaction from level-%d to level-%d from %s .. "
                  "%s; will stop at %s\n",
                  m->cfd->GetName().c_str(), m->input_level, c->output_level(),
                  (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                  (m->end ? m->end->DebugString().c_str() : "(end)"),
                  ((m->done || manual_end == nullptr)
                       ? "(end)"
                       : manual_end->DebugString().c_str()));
G
Gabor Cselle 已提交
2259
    }
2260 2261 2262 2263 2264 2265 2266 2267 2268 2269 2270 2271 2272 2273 2274 2275 2276 2277 2278 2279 2280 2281 2282 2283 2284 2285 2286 2287 2288 2289 2290 2291 2292 2293 2294 2295 2296 2297 2298 2299 2300 2301 2302 2303 2304 2305 2306
  } else if (!compaction_queue_.empty()) {
    // cfd is referenced here
    auto cfd = PopFirstFromCompactionQueue();
    // We unreference here because the following code will take a Ref() on
    // this cfd if it is going to use it (Compaction class holds a
    // reference).
    // This will all happen under a mutex so we don't have to be afraid of
    // somebody else deleting it.
    if (cfd->Unref()) {
      delete cfd;
      // This was the last reference of the column family, so no need to
      // compact.
      return Status::OK();
    }

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

2312
  if (!c) {
H
hans@chromium.org 已提交
2313
    // Nothing to do
2314
    LogToBuffer(log_buffer, "Compaction nothing to do");
I
Igor Canadi 已提交
2315 2316 2317 2318 2319
  } else if (c->IsDeletionCompaction()) {
    // TODO(icanadi) Do we want to honor snapshots here? i.e. not delete old
    // file if there is alive snapshot pointing to it
    assert(c->num_input_files(1) == 0);
    assert(c->level() == 0);
2320
    assert(c->column_family_data()->ioptions()->compaction_style ==
I
Igor Canadi 已提交
2321 2322
           kCompactionStyleFIFO);
    for (const auto& f : *c->inputs(0)) {
2323
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
I
Igor Canadi 已提交
2324
    }
2325 2326 2327
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
2328 2329
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
I
Igor Canadi 已提交
2330 2331 2332 2333
    LogToBuffer(log_buffer, "[%s] Deleted %d files\n",
                c->column_family_data()->GetName().c_str(),
                c->num_input_files(0));
    *madeProgress = true;
H
hans@chromium.org 已提交
2334
  } else if (!is_manual && c->IsTrivialMove()) {
2335
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2336 2337 2338 2339 2340
    // Instrument for event update
    // TODO(yhchiang): add op details for showing trivial-move.
    ThreadStatusUtil::SetColumnFamily(c->column_family_data());
    ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);

J
jorlow@chromium.org 已提交
2341
    // Move file to next level
2342
    assert(c->num_input_files(0) == 1);
J
jorlow@chromium.org 已提交
2343
    FileMetaData* f = c->input(0, 0);
2344
    c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
S
sdong 已提交
2345
    c->edit()->AddFile(c->output_level(), f->fd.GetNumber(), f->fd.GetPathId(),
2346 2347
                       f->fd.GetFileSize(), f->smallest, f->largest,
                       f->smallest_seqno, f->largest_seqno);
2348
    status = versions_->LogAndApply(c->column_family_data(),
2349 2350
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2351
    // Use latest MutableCFOptions
I
Igor Canadi 已提交
2352 2353
    InstallSuperVersionBackground(c->column_family_data(), job_context,
                                  *c->mutable_cf_options());
2354

S
sdong 已提交
2355
    VersionStorageInfo::LevelSummaryStorage tmp;
2356 2357
    c->column_family_data()->internal_stats()->IncBytesMoved(
        c->level() + 1, f->fd.GetFileSize());
2358 2359 2360 2361 2362 2363 2364
    {
      event_logger_.LogToBuffer(log_buffer)
          << "job" << job_context->job_id << "event"
          << "trivial_move"
          << "destination_level" << c->level() + 1 << "file_number"
          << f->fd.GetNumber() << "file_size" << f->fd.GetFileSize();
    }
2365 2366 2367 2368 2369 2370
    LogToBuffer(
        log_buffer,
        "[%s] Moved #%" PRIu64 " to level-%d %" PRIu64 " bytes %s: %s\n",
        c->column_family_data()->GetName().c_str(), f->fd.GetNumber(),
        c->level() + 1, f->fd.GetFileSize(), status.ToString().c_str(),
        c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
2371
    *madeProgress = true;
2372 2373 2374

    // Clear Instrument
    ThreadStatusUtil::ResetThreadStatus();
J
jorlow@chromium.org 已提交
2375
  } else {
2376
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial");
I
Igor Canadi 已提交
2377 2378 2379 2380 2381
    auto yield_callback = [&]() {
      return CallFlushDuringCompaction(c->column_family_data(),
                                       *c->mutable_cf_options(), job_context,
                                       log_buffer);
    };
I
Igor Canadi 已提交
2382
    assert(is_snapshot_supported_ || snapshots_.empty());
2383
    CompactionJob compaction_job(
I
Igor Canadi 已提交
2384 2385 2386 2387 2388
        job_context->job_id, c.get(), db_options_, env_options_,
        versions_.get(), &shutting_down_, log_buffer, directories_.GetDbDir(),
        directories_.GetDataDir(c->GetOutputPathId()), stats_,
        snapshots_.GetAll(), table_cache_, std::move(yield_callback),
        &event_logger_, c->mutable_cf_options()->paranoid_file_checks);
I
Igor Canadi 已提交
2389 2390 2391 2392
    compaction_job.Prepare();
    mutex_.Unlock();
    status = compaction_job.Run();
    mutex_.Lock();
I
Igor Canadi 已提交
2393
    compaction_job.Install(&status, *c->mutable_cf_options(), &mutex_);
I
Igor Canadi 已提交
2394 2395 2396 2397
    if (status.ok()) {
      InstallSuperVersionBackground(c->column_family_data(), job_context,
                                    *c->mutable_cf_options());
    }
O
Ori Bernstein 已提交
2398 2399 2400 2401 2402
    *madeProgress = true;
  }
  // FIXME(orib): should I check if column family data is null?
  if (c != nullptr) {
    NotifyOnCompactionCompleted(c->column_family_data(), c.get(), status);
I
Igor Canadi 已提交
2403
    c->ReleaseCompactionFiles(status);
2404
    *madeProgress = true;
J
jorlow@chromium.org 已提交
2405
  }
2406
  // this will unref its input_version and column_family_data
2407
  c.reset();
J
jorlow@chromium.org 已提交
2408 2409 2410

  if (status.ok()) {
    // Done
F
Feng Zhu 已提交
2411
  } else if (status.IsShutdownInProgress()) {
J
jorlow@chromium.org 已提交
2412 2413
    // Ignore compaction errors found during shutting down
  } else {
2414
    Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log, "Compaction error: %s",
2415
        status.ToString().c_str());
2416
    if (db_options_.paranoid_checks && bg_error_.ok()) {
J
jorlow@chromium.org 已提交
2417 2418 2419
      bg_error_ = status;
    }
  }
H
hans@chromium.org 已提交
2420 2421

  if (is_manual) {
G
Gabor Cselle 已提交
2422
    ManualCompaction* m = manual_compaction_;
2423
    if (!status.ok()) {
L
Lei Jin 已提交
2424
      m->status = status;
2425 2426
      m->done = true;
    }
2427 2428 2429 2430 2431 2432 2433 2434 2435
    // 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.
2436 2437 2438 2439 2440
    //
    // Stop the compaction if manual_end points to nullptr -- this means
    // that we compacted the whole range. manual_end should always point
    // to nullptr in case of universal compaction
    if (manual_end == nullptr) {
2441 2442
      m->done = true;
    }
G
Gabor Cselle 已提交
2443 2444 2445
    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 已提交
2446
      // Universal and FIFO compactions should always compact the whole range
S
sdong 已提交
2447 2448 2449
      assert(m->cfd->ioptions()->compaction_style !=
                 kCompactionStyleUniversal ||
             m->cfd->ioptions()->num_levels > 1);
2450
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
2451
      m->tmp_storage = *manual_end;
G
Gabor Cselle 已提交
2452 2453
      m->begin = &m->tmp_storage;
    }
2454
    m->in_progress = false; // not being processed anymore
2455
    manual_compaction_ = nullptr;
H
hans@chromium.org 已提交
2456
  }
2457
  return status;
J
jorlow@chromium.org 已提交
2458 2459
}

I
Igor Canadi 已提交
2460 2461 2462
uint64_t DBImpl::CallFlushDuringCompaction(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
    JobContext* job_context, LogBuffer* log_buffer) {
2463
  if (db_options_.max_background_flushes > 0) {
2464 2465 2466
    // flush thread will take care of this
    return 0;
  }
I
Igor Canadi 已提交
2467
  if (cfd->imm()->imm_flush_needed.load(std::memory_order_relaxed)) {
I
Igor Canadi 已提交
2468 2469 2470 2471
    const uint64_t imm_start = env_->NowMicros();
    mutex_.Lock();
    if (cfd->imm()->IsFlushPending()) {
      cfd->Ref();
I
Igor Canadi 已提交
2472 2473
      FlushMemTableToOutputFile(cfd, mutable_cf_options, nullptr, job_context,
                                log_buffer);
I
Igor Canadi 已提交
2474
      cfd->Unref();
I
Igor Canadi 已提交
2475
      bg_cv_.SignalAll();  // Wakeup DelayWrite() if necessary
I
Igor Canadi 已提交
2476 2477 2478 2479 2480 2481 2482 2483
    }
    mutex_.Unlock();
    log_buffer->FlushBufferToLog();
    return env_->NowMicros() - imm_start;
  }
  return 0;
}

2484 2485
namespace {
struct IterState {
2486
  IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version)
I
Igor Canadi 已提交
2487
      : db(_db), mu(_mu), super_version(_super_version) {}
2488 2489

  DBImpl* db;
2490
  InstrumentedMutex* mu;
2491
  SuperVersion* super_version;
2492 2493 2494 2495
};

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

2497
  if (state->super_version->Unref()) {
2498 2499 2500
    // Job id == 0 means that this is not our background process, but rather
    // user thread
    JobContext job_context(0);
2501

2502 2503
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
2504
    state->db->FindObsoleteFiles(&job_context, false, true);
2505 2506 2507
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
2508 2509
    if (job_context.HaveSomethingToDelete()) {
      state->db->PurgeObsoleteFiles(job_context);
2510
    }
I
Igor Canadi 已提交
2511
    job_context.Clean();
I
Igor Canadi 已提交
2512
  }
T
Tomislav Novak 已提交
2513

2514 2515
  delete state;
}
H
Hans Wennborg 已提交
2516
}  // namespace
2517

L
Lei Jin 已提交
2518
Iterator* DBImpl::NewInternalIterator(const ReadOptions& read_options,
2519
                                      ColumnFamilyData* cfd,
2520 2521 2522
                                      SuperVersion* super_version,
                                      Arena* arena) {
  Iterator* internal_iter;
2523 2524 2525 2526 2527
  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 已提交
2528
      super_version->mem->NewIterator(read_options, arena));
2529
  // Collect all needed child iterators for immutable memtables
L
Lei Jin 已提交
2530
  super_version->imm->AddIterators(read_options, &merge_iter_builder);
2531
  // Collect iterators for files in L0 - Ln
L
Lei Jin 已提交
2532
  super_version->current->AddIterators(read_options, env_options_,
2533 2534
                                       &merge_iter_builder);
  internal_iter = merge_iter_builder.Finish();
2535
  IterState* cleanup = new IterState(this, &mutex_, super_version);
2536
  internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);
J
jorlow@chromium.org 已提交
2537 2538 2539 2540

  return internal_iter;
}

2541 2542 2543 2544
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
2545
Status DBImpl::Get(const ReadOptions& read_options,
2546
                   ColumnFamilyHandle* column_family, const Slice& key,
J
jorlow@chromium.org 已提交
2547
                   std::string* value) {
L
Lei Jin 已提交
2548
  return GetImpl(read_options, column_family, key, value);
2549 2550
}

I
Igor Canadi 已提交
2551 2552
// JobContext gets created and destructed outside of the lock --
// we
I
Igor Canadi 已提交
2553 2554
// use this convinently to:
// * malloc one SuperVersion() outside of the lock -- new_superversion
2555
// * delete SuperVersion()s outside of the lock -- superversions_to_free
I
Igor Canadi 已提交
2556
//
I
Igor Canadi 已提交
2557 2558 2559 2560
// However, if InstallSuperVersion() gets called twice with the same
// job_context, we can't reuse the SuperVersion() that got
// malloced
// because
I
Igor Canadi 已提交
2561 2562 2563
// 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 已提交
2564 2565
void DBImpl::InstallSuperVersionBackground(
    ColumnFamilyData* cfd, JobContext* job_context,
2566
    const MutableCFOptions& mutable_cf_options) {
2567
  mutex_.AssertHeld();
I
Igor Canadi 已提交
2568 2569 2570 2571
  SuperVersion* old_superversion = InstallSuperVersion(
      cfd, job_context->new_superversion, mutable_cf_options);
  job_context->new_superversion = nullptr;
  job_context->superversions_to_free.push_back(old_superversion);
I
Igor Canadi 已提交
2572 2573
}

L
Lei Jin 已提交
2574 2575
SuperVersion* DBImpl::InstallSuperVersion(
    ColumnFamilyData* cfd, SuperVersion* new_sv,
2576
    const MutableCFOptions& mutable_cf_options, bool dont_schedule_bg_work) {
L
Lei Jin 已提交
2577
  mutex_.AssertHeld();
2578 2579 2580 2581 2582 2583 2584 2585 2586

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

2590 2591 2592 2593 2594 2595 2596 2597 2598
  // Whenever we install new SuperVersion, we might need to issue new flushes or
  // compactions. dont_schedule_bg_work is true when scheduling from write
  // thread and we don't want to add additional overhead. Callers promise to
  // call SchedulePendingFlush() and MaybeScheduleFlushOrCompaction() eventually
  if (!dont_schedule_bg_work) {
    SchedulePendingFlush(cfd);
    SchedulePendingCompaction(cfd);
    MaybeScheduleFlushOrCompaction();
  }
L
Lei Jin 已提交
2599 2600 2601 2602 2603 2604 2605 2606 2607 2608

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

2614 2615 2616
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2617
  SequenceNumber snapshot;
L
Lei Jin 已提交
2618 2619 2620
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2621 2622
  } else {
    snapshot = versions_->LastSequence();
J
jorlow@chromium.org 已提交
2623
  }
2624

2625
  // Acquire SuperVersion
2626
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
I
Igor Canadi 已提交
2627

2628
  // Prepare to store a list of merge operations if merge occurs.
2629
  MergeContext merge_context;
2630

2631
  Status s;
2632
  // First look in the memtable, then in the immutable memtable (if any).
2633
  // s is both in/out. When in, s could either be OK or MergeInProgress.
2634
  // merge_operands will contain the sequence of merges in the latter case.
2635
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
2636
  PERF_TIMER_STOP(get_snapshot_time);
2637

2638
  if (sv->mem->Get(lkey, value, &s, &merge_context)) {
2639
    // Done
L
Lei Jin 已提交
2640
    RecordTick(stats_, MEMTABLE_HIT);
2641
  } else if (sv->imm->Get(lkey, value, &s, &merge_context)) {
2642
    // Done
L
Lei Jin 已提交
2643
    RecordTick(stats_, MEMTABLE_HIT);
2644
  } else {
2645
    PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2646 2647
    sv->current->Get(read_options, lkey, value, &s, &merge_context,
                     value_found);
L
Lei Jin 已提交
2648
    RecordTick(stats_, MEMTABLE_MISS);
2649
  }
2650

2651 2652
  {
    PERF_TIMER_GUARD(get_post_process_time);
2653

2654
    ReturnAndCleanupSuperVersion(cfd, sv);
2655

2656 2657 2658
    RecordTick(stats_, NUMBER_KEYS_READ);
    RecordTick(stats_, BYTES_READ, value->size());
  }
2659
  return s;
J
jorlow@chromium.org 已提交
2660 2661
}

2662
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
2663
    const ReadOptions& read_options,
2664
    const std::vector<ColumnFamilyHandle*>& column_family,
2665
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
2666

L
Lei Jin 已提交
2667
  StopWatch sw(env_, stats_, DB_MULTIGET);
2668
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
2669

2670
  SequenceNumber snapshot;
2671

2672
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
2673
    ColumnFamilyData* cfd;
2674 2675 2676 2677 2678
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
2679 2680 2681 2682 2683 2684
    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});
2685 2686 2687
    }
  }

2688
  mutex_.Lock();
L
Lei Jin 已提交
2689 2690 2691
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
2692 2693 2694
  } else {
    snapshot = versions_->LastSequence();
  }
2695
  for (auto mgd_iter : multiget_cf_data) {
2696 2697
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
2698
  }
2699
  mutex_.Unlock();
2700

2701 2702
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
2703

2704
  // Note: this always resizes the values array
2705 2706 2707
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
2708 2709

  // Keep track of bytes that we read for statistics-recording later
2710
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
2711
  PERF_TIMER_STOP(get_snapshot_time);
2712 2713 2714 2715

  // 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.
2716
  // merge_operands will contain the sequence of merges in the latter case.
2717
  for (size_t i = 0; i < num_keys; ++i) {
2718
    merge_context.Clear();
2719
    Status& s = stat_list[i];
2720 2721 2722
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
2723 2724
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
2725 2726 2727
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
2728
    if (super_version->mem->Get(lkey, value, &s, &merge_context)) {
2729
      // Done
2730
    } else if (super_version->imm->Get(lkey, value, &s, &merge_context)) {
2731 2732
      // Done
    } else {
2733
      PERF_TIMER_GUARD(get_from_output_files_time);
L
Lei Jin 已提交
2734 2735
      super_version->current->Get(read_options, lkey, value, &s,
                                  &merge_context);
2736 2737 2738
    }

    if (s.ok()) {
2739
      bytes_read += value->size();
2740 2741 2742 2743
    }
  }

  // Post processing (decrement reference counts and record statistics)
2744
  PERF_TIMER_GUARD(get_post_process_time);
2745 2746
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
2747
  // TODO(icanadi) do we need lock here or just around Cleanup()?
2748 2749 2750 2751 2752 2753
  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);
2754 2755
    }
  }
2756 2757 2758 2759 2760 2761 2762
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
2763
  }
2764

L
Lei Jin 已提交
2765 2766 2767
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
L
Lei Jin 已提交
2768
  PERF_TIMER_STOP(get_post_process_time);
2769

2770
  return stat_list;
2771 2772
}

L
Lei Jin 已提交
2773
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
2774
                                  const std::string& column_family_name,
2775
                                  ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
2776
  Status s;
I
Igor Canadi 已提交
2777
  *handle = nullptr;
Y
Yueh-Hsuan Chiang 已提交
2778
  {
2779
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
2780

Y
Yueh-Hsuan Chiang 已提交
2781 2782 2783 2784 2785 2786 2787 2788 2789 2790 2791 2792 2793 2794
    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 已提交
2795 2796 2797 2798 2799 2800 2801 2802
    {  // write thread
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
      // LogAndApply will both write the creation in MANIFEST and create
      // ColumnFamilyData object
      s = versions_->LogAndApply(
          nullptr, MutableCFOptions(opt, ImmutableCFOptions(opt)), &edit,
2803
          &mutex_, directories_.GetDbDir(), false, &cf_options);
I
Igor Canadi 已提交
2804 2805
      write_thread_.ExitWriteThread(&w, &w, s);
    }
Y
Yueh-Hsuan Chiang 已提交
2806 2807 2808 2809 2810 2811 2812
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
      delete InstallSuperVersion(
          cfd, nullptr, *cfd->GetLatestMutableCFOptions());
S
sdong 已提交
2813 2814 2815 2816 2817

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

Y
Yueh-Hsuan Chiang 已提交
2818 2819 2820 2821 2822 2823 2824 2825 2826
      *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());
    }
2827
  }  // InstrumentedMutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
2828 2829

  // this is outside the mutex
2830
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2831 2832
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
2833
  }
2834
  return s;
2835 2836
}

2837 2838 2839 2840
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
2841 2842
    return Status::InvalidArgument("Can't drop default column family");
  }
2843

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

I
Igor Canadi 已提交
2846 2847
  VersionEdit edit;
  edit.DropColumnFamily();
2848 2849
  edit.SetColumnFamily(cfd->GetID());

2850
  Status s;
2851
  {
2852
    InstrumentedMutexLock l(&mutex_);
2853 2854 2855 2856
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
2857
      // we drop column family from a single write thread
I
Igor Canadi 已提交
2858 2859
      WriteThread::Writer w(&mutex_);
      s = write_thread_.EnterWriteThread(&w, 0);
2860
      assert(s.ok() && !w.done);  // No timeout and nobody should do our job
2861 2862
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
I
Igor Canadi 已提交
2863
      write_thread_.ExitWriteThread(&w, &w, s);
2864
    }
S
sdong 已提交
2865 2866 2867 2868 2869 2870

    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()) {
2871
        if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) {
S
sdong 已提交
2872 2873 2874 2875 2876 2877
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
2878
  }
2879

2880
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
2881 2882 2883 2884
    // 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 已提交
2885
    assert(cfd->IsDropped());
L
Lei Jin 已提交
2886 2887 2888
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                  mutable_cf_options->max_write_buffer_number;
2889 2890
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Dropped column family with id %u\n",
2891
        cfd->GetID());
2892
  } else {
2893
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
2894
        "Dropping column family with id %u FAILED -- %s\n",
2895 2896 2897
        cfd->GetID(), s.ToString().c_str());
  }

2898
  return s;
2899 2900
}

L
Lei Jin 已提交
2901
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
2902 2903
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
2904
  if (value_found != nullptr) {
K
Kai Liu 已提交
2905 2906
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
2907
  }
L
Lei Jin 已提交
2908
  ReadOptions roptions = read_options;
2909
  roptions.read_tier = kBlockCacheTier; // read from block cache only
2910
  auto s = GetImpl(roptions, column_family, key, value, value_found);
K
Kai Liu 已提交
2911

2912
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
2913 2914 2915
  // 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();
2916 2917
}

2918
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
2919 2920 2921
                              ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
T
Tomislav Novak 已提交
2922

V
Venkatesh Radhakrishnan 已提交
2923 2924 2925 2926 2927 2928 2929 2930 2931 2932 2933 2934 2935 2936 2937 2938 2939 2940
  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 已提交
2941 2942 2943 2944
#ifdef ROCKSDB_LITE
    // not supported in lite version
    return nullptr;
#else
2945 2946
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
2947
    return NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
2948 2949 2950
        kMaxSequenceNumber,
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_options.iterate_upper_bound);
I
Igor Canadi 已提交
2951
#endif
T
Tomislav Novak 已提交
2952
  } else {
2953
    SequenceNumber latest_snapshot = versions_->LastSequence();
2954
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
2955

I
Igor Canadi 已提交
2956
    auto snapshot =
2957 2958 2959
        read_options.snapshot != nullptr
            ? reinterpret_cast<const SnapshotImpl*>(
                read_options.snapshot)->number_
I
Igor Canadi 已提交
2960
            : latest_snapshot;
T
Tomislav Novak 已提交
2961

2962 2963 2964 2965 2966 2967 2968 2969 2970 2971 2972 2973 2974 2975 2976 2977 2978 2979 2980 2981 2982 2983 2984 2985 2986 2987 2988 2989 2990 2991 2992 2993 2994 2995 2996 2997 2998 2999 3000 3001 3002 3003 3004
    // Try to generate a DB iterator tree in continuous memory area to be
    // cache friendly. Here is an example of result:
    // +-------------------------------+
    // |                               |
    // | ArenaWrappedDBIter            |
    // |  +                            |
    // |  +---> Inner Iterator   ------------+
    // |  |                            |     |
    // |  |    +-- -- -- -- -- -- -- --+     |
    // |  +--- | Arena                 |     |
    // |       |                       |     |
    // |          Allocated Memory:    |     |
    // |       |   +-------------------+     |
    // |       |   | DBIter            | <---+
    // |           |  +                |
    // |       |   |  +-> iter_  ------------+
    // |       |   |                   |     |
    // |       |   +-------------------+     |
    // |       |   | MergingIterator   | <---+
    // |           |  +                |
    // |       |   |  +->child iter1  ------------+
    // |       |   |  |                |          |
    // |           |  +->child iter2  ----------+ |
    // |       |   |  |                |        | |
    // |       |   |  +->child iter3  --------+ | |
    // |           |                   |      | | |
    // |       |   +-------------------+      | | |
    // |       |   | Iterator1         | <--------+
    // |       |   +-------------------+      | |
    // |       |   | Iterator2         | <------+
    // |       |   +-------------------+      |
    // |       |   | Iterator3         | <----+
    // |       |   +-------------------+
    // |       |                       |
    // +-------+-----------------------+
    //
    // ArenaWrappedDBIter inlines an arena area where all the iterartor in the
    // the iterator tree is allocated in the order of being accessed when
    // querying.
    // Laying out the iterators in the order of being accessed makes it more
    // likely that any iterator pointer is close to the iterator it points to so
    // that they are likely to be in the same cache line and/or page.
    ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3005
        env_, *cfd->ioptions(), cfd->user_comparator(),
3006
        snapshot, sv->mutable_cf_options.max_sequential_skip_in_iterations,
3007
        read_options.iterate_upper_bound);
3008

3009
    Iterator* internal_iter =
3010
        NewInternalIterator(read_options, cfd, sv, db_iter->GetArena());
3011 3012 3013 3014
    db_iter->SetIterUnderDBIter(internal_iter);

    return db_iter;
  }
3015 3016
  // To stop compiler from complaining
  return nullptr;
J
jorlow@chromium.org 已提交
3017 3018
}

3019
Status DBImpl::NewIterators(
3020
    const ReadOptions& read_options,
I
Igor Canadi 已提交
3021
    const std::vector<ColumnFamilyHandle*>& column_families,
3022
    std::vector<Iterator*>* iterators) {
I
Igor Canadi 已提交
3023 3024
  iterators->clear();
  iterators->reserve(column_families.size());
V
Venkatesh Radhakrishnan 已提交
3025 3026 3027 3028 3029 3030 3031 3032 3033 3034 3035 3036 3037 3038 3039 3040 3041 3042 3043 3044
  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 已提交
3045 3046 3047 3048
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
I
Igor Canadi 已提交
3049 3050
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
3051 3052
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
L
Lei Jin 已提交
3053
      iterators->push_back(
3054
          NewDBIterator(env_, *cfd->ioptions(), cfd->user_comparator(), iter,
3055 3056
              kMaxSequenceNumber,
              sv->mutable_cf_options.max_sequential_skip_in_iterations));
I
Igor Canadi 已提交
3057
    }
I
Igor Canadi 已提交
3058
#endif
I
Igor Canadi 已提交
3059
  } else {
3060 3061
    SequenceNumber latest_snapshot = versions_->LastSequence();

I
Igor Canadi 已提交
3062
    for (size_t i = 0; i < column_families.size(); ++i) {
3063 3064 3065
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
I
Igor Canadi 已提交
3066 3067

      auto snapshot =
3068 3069 3070
          read_options.snapshot != nullptr
              ? reinterpret_cast<const SnapshotImpl*>(
                  read_options.snapshot)->number_
I
Igor Canadi 已提交
3071 3072
              : latest_snapshot;

3073
      ArenaWrappedDBIter* db_iter = NewArenaWrappedDbIterator(
3074
          env_, *cfd->ioptions(), cfd->user_comparator(), snapshot,
3075
          sv->mutable_cf_options.max_sequential_skip_in_iterations);
3076
      Iterator* internal_iter = NewInternalIterator(
3077
          read_options, cfd, sv, db_iter->GetArena());
3078 3079
      db_iter->SetIterUnderDBIter(internal_iter);
      iterators->push_back(db_iter);
I
Igor Canadi 已提交
3080 3081 3082 3083
    }
  }

  return Status::OK();
3084 3085
}

J
jorlow@chromium.org 已提交
3086
const Snapshot* DBImpl::GetSnapshot() {
3087 3088 3089
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error

3090
  InstrumentedMutexLock l(&mutex_);
3091
  // returns null if the underlying memtable does not support snapshot.
S
sdong 已提交
3092
  if (!is_snapshot_supported_) return nullptr;
3093
  return snapshots_.New(versions_->LastSequence(), unix_time);
J
jorlow@chromium.org 已提交
3094 3095 3096
}

void DBImpl::ReleaseSnapshot(const Snapshot* s) {
3097
  InstrumentedMutexLock l(&mutex_);
3098
  snapshots_.Delete(reinterpret_cast<const SnapshotImpl*>(s));
J
jorlow@chromium.org 已提交
3099 3100 3101
}

// Convenience methods
3102 3103
Status DBImpl::Put(const WriteOptions& o, ColumnFamilyHandle* column_family,
                   const Slice& key, const Slice& val) {
3104
  return DB::Put(o, column_family, key, val);
J
jorlow@chromium.org 已提交
3105 3106
}

3107 3108 3109
Status DBImpl::Merge(const WriteOptions& o, ColumnFamilyHandle* column_family,
                     const Slice& key, const Slice& val) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
3110
  if (!cfh->cfd()->ioptions()->merge_operator) {
3111 3112
    return Status::NotSupported("Provide a merge_operator when opening DB");
  } else {
3113
    return DB::Merge(o, column_family, key, val);
3114 3115 3116
  }
}

L
Lei Jin 已提交
3117
Status DBImpl::Delete(const WriteOptions& write_options,
3118
                      ColumnFamilyHandle* column_family, const Slice& key) {
L
Lei Jin 已提交
3119
  return DB::Delete(write_options, column_family, key);
J
jorlow@chromium.org 已提交
3120 3121
}

L
Lei Jin 已提交
3122
Status DBImpl::Write(const WriteOptions& write_options, WriteBatch* my_batch) {
S
Stanislau Hlebik 已提交
3123 3124 3125
  if (my_batch == nullptr) {
    return Status::Corruption("Batch is nullptr!");
  }
3126
  PERF_TIMER_GUARD(write_pre_and_post_process_time);
I
Igor Canadi 已提交
3127
  WriteThread::Writer w(&mutex_);
S
Stanislau Hlebik 已提交
3128
  w.batch = my_batch;
L
Lei Jin 已提交
3129 3130
  w.sync = write_options.sync;
  w.disableWAL = write_options.disableWAL;
S
Stanislau Hlebik 已提交
3131 3132
  w.in_batch_group = false;
  w.done = false;
L
Lei Jin 已提交
3133
  w.timeout_hint_us = write_options.timeout_hint_us;
S
Stanislau Hlebik 已提交
3134 3135

  uint64_t expiration_time = 0;
I
Igor Canadi 已提交
3136
  bool has_timeout = false;
S
Stanislau Hlebik 已提交
3137
  if (w.timeout_hint_us == 0) {
I
Igor Canadi 已提交
3138
    w.timeout_hint_us = WriteThread::kNoTimeOut;
S
Stanislau Hlebik 已提交
3139 3140
  } else {
    expiration_time = env_->NowMicros() + w.timeout_hint_us;
I
Igor Canadi 已提交
3141
    has_timeout = true;
S
Stanislau Hlebik 已提交
3142 3143
  }

L
Lei Jin 已提交
3144
  if (!write_options.disableWAL) {
S
Stanislau Hlebik 已提交
3145 3146 3147 3148 3149
    RecordTick(stats_, WRITE_WITH_WAL);
  }

  WriteContext context;
  mutex_.Lock();
I
Igor Canadi 已提交
3150 3151 3152 3153 3154

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

I
Igor Canadi 已提交
3155
  Status status = write_thread_.EnterWriteThread(&w, expiration_time);
S
Stanislau Hlebik 已提交
3156 3157
  assert(status.ok() || status.IsTimedOut());
  if (status.IsTimedOut()) {
3158
    mutex_.Unlock();
L
Lei Jin 已提交
3159
    RecordTick(stats_, WRITE_TIMEDOUT);
3160
    return Status::TimedOut();
3161
  }
S
Stanislau Hlebik 已提交
3162 3163 3164 3165 3166 3167 3168 3169 3170 3171
  if (w.done) {  // write was done by someone else
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_OTHER,
                                           1);
    mutex_.Unlock();
    RecordTick(stats_, WRITE_DONE_BY_OTHER);
    return w.status;
  }

  RecordTick(stats_, WRITE_DONE_BY_SELF);
  default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_DONE_BY_SELF, 1);
3172

3173 3174 3175 3176
  // 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.
3177 3178 3179
  assert(!single_column_family_mode_ ||
         versions_->GetColumnFamilySet()->NumberOfColumnFamilies() == 1);

3180
  uint64_t max_total_wal_size = (db_options_.max_total_wal_size == 0)
3181
                                    ? 4 * max_total_in_memory_state_
3182
                                    : db_options_.max_total_wal_size;
3183
  if (UNLIKELY(!single_column_family_mode_) &&
3184
      alive_log_files_.begin()->getting_flushed == false &&
I
Igor Canadi 已提交
3185
      total_log_size_ > max_total_wal_size) {
I
Igor Canadi 已提交
3186
    uint64_t flush_column_family_if_log_file = alive_log_files_.begin()->number;
I
Igor Canadi 已提交
3187
    alive_log_files_.begin()->getting_flushed = true;
3188
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3189 3190 3191
        "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 已提交
3192 3193
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
3194
    for (auto cfd : *versions_->GetColumnFamilySet()) {
3195 3196 3197
      if (cfd->IsDropped()) {
        continue;
      }
I
Igor Canadi 已提交
3198
      if (cfd->GetLogNumber() <= flush_column_family_if_log_file) {
S
Stanislau Hlebik 已提交
3199
        status = SetNewMemtableAndNewLogFile(cfd, &context);
I
Igor Canadi 已提交
3200 3201 3202
        if (!status.ok()) {
          break;
        }
S
Stanislau Hlebik 已提交
3203
        cfd->imm()->FlushRequested();
3204 3205
        SchedulePendingFlush(cfd);
        context.schedule_bg_work_ = true;
3206
      }
3207
    }
3208 3209 3210 3211 3212 3213 3214 3215
  } else if (UNLIKELY(write_buffer_.ShouldFlush())) {
    Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
        "Flushing all column families. Write buffer is using %" PRIu64
        " bytes out of a total of %" PRIu64 ".",
        write_buffer_.memory_usage(), write_buffer_.buffer_size());
    // no need to refcount because drop is happening in write thread, so can't
    // happen while we're in the write thread
    for (auto cfd : *versions_->GetColumnFamilySet()) {
3216 3217 3218
      if (cfd->IsDropped()) {
        continue;
      }
3219 3220 3221 3222 3223 3224
      if (!cfd->mem()->IsEmpty()) {
        status = SetNewMemtableAndNewLogFile(cfd, &context);
        if (!status.ok()) {
          break;
        }
        cfd->imm()->FlushRequested();
3225 3226
        SchedulePendingFlush(cfd);
        context.schedule_bg_work_ = true;
3227 3228 3229
      }
    }
    MaybeScheduleFlushOrCompaction();
I
Igor Canadi 已提交
3230 3231 3232 3233 3234 3235 3236 3237 3238 3239
  }

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

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

I
Igor Canadi 已提交
3240 3241
  if (UNLIKELY(status.ok() && (write_controller_.IsStopped() ||
                               write_controller_.GetDelay() > 0))) {
3242 3243 3244 3245 3246
    // If writer is stopped, we need to get it going,
    // so schedule flushes/compactions
    if (context.schedule_bg_work_) {
      MaybeScheduleFlushOrCompaction();
    }
I
Igor Canadi 已提交
3247
    status = DelayWrite(expiration_time);
I
Igor Canadi 已提交
3248 3249 3250 3251 3252
  }

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

D
dgrogan@chromium.org 已提交
3255
  uint64_t last_sequence = versions_->LastSequence();
I
Igor Canadi 已提交
3256
  WriteThread::Writer* last_writer = &w;
S
Stanislau Hlebik 已提交
3257
  if (status.ok()) {
3258
    autovector<WriteBatch*> write_batch_group;
I
Igor Canadi 已提交
3259
    write_thread_.BuildBatchGroup(&last_writer, &write_batch_group);
3260

3261 3262 3263
    // 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
3264
    // into memtables
3265
    {
3266
      mutex_.Unlock();
3267 3268 3269 3270 3271 3272 3273 3274 3275 3276
      WriteBatch* updates = nullptr;
      if (write_batch_group.size() == 1) {
        updates = write_batch_group[0];
      } else {
        updates = &tmp_batch_;
        for (size_t i = 0; i < write_batch_group.size(); ++i) {
          WriteBatchInternal::Append(updates, write_batch_group[i]);
        }
      }

3277 3278 3279 3280
      const SequenceNumber current_sequence = last_sequence + 1;
      WriteBatchInternal::SetSequence(updates, current_sequence);
      int my_batch_count = WriteBatchInternal::Count(updates);
      last_sequence += my_batch_count;
3281
      const uint64_t batch_size = WriteBatchInternal::ByteSize(updates);
3282
      // Record statistics
L
Lei Jin 已提交
3283
      RecordTick(stats_, NUMBER_KEYS_WRITTEN, my_batch_count);
S
sdong 已提交
3284
      RecordTick(stats_, BYTES_WRITTEN, batch_size);
L
Lei Jin 已提交
3285
      if (write_options.disableWAL) {
3286
        flush_on_destroy_ = true;
3287
      }
L
Lei Jin 已提交
3288
      PERF_TIMER_STOP(write_pre_and_post_process_time);
3289

3290
      uint64_t log_size = 0;
L
Lei Jin 已提交
3291
      if (!write_options.disableWAL) {
3292
        PERF_TIMER_GUARD(write_wal_time);
3293 3294
        Slice log_entry = WriteBatchInternal::Contents(updates);
        status = log_->AddRecord(log_entry);
I
Igor Canadi 已提交
3295 3296
        total_log_size_ += log_entry.size();
        alive_log_files_.back().AddSize(log_entry.size());
3297
        log_empty_ = false;
3298
        log_size = log_entry.size();
L
Lei Jin 已提交
3299
        RecordTick(stats_, WAL_FILE_BYTES, log_size);
L
Lei Jin 已提交
3300
        if (status.ok() && write_options.sync) {
I
Igor Canadi 已提交
3301
          RecordTick(stats_, WAL_FILE_SYNCED);
3302
          StopWatch sw(env_, stats_, WAL_FILE_SYNC_MICROS);
3303
          if (db_options_.use_fsync) {
3304
            status = log_->file()->Fsync();
3305
          } else {
3306
            status = log_->file()->Sync();
3307
          }
3308
          if (status.ok() && !log_dir_synced_) {
3309 3310 3311 3312 3313
            // 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();
          }
3314
          log_dir_synced_ = true;
H
heyongqiang 已提交
3315
        }
3316 3317
      }
      if (status.ok()) {
3318
        PERF_TIMER_GUARD(write_memtable_time);
Y
Yueh-Hsuan Chiang 已提交
3319

3320
        status = WriteBatchInternal::InsertInto(
I
Igor Canadi 已提交
3321
            updates, column_family_memtables_.get(),
L
Lei Jin 已提交
3322
            write_options.ignore_missing_column_families, 0, this, false);
Y
Yueh-Hsuan Chiang 已提交
3323 3324 3325 3326 3327 3328 3329 3330
        // A non-OK status here indicates iteration failure (either in-memory
        // writebatch corruption (very bad), or the client specified invalid
        // column family).  This will later on trigger bg_error_.
        //
        // Note that existing logic was not sound. Any partial failure writing
        // into the memtable would result in a state that some write ops might
        // have succeeded in memtable but Status reports error for all writes.

L
Lei Jin 已提交
3331
        SetTickerCount(stats_, SEQUENCE_NUMBER, last_sequence);
3332
      }
L
Lei Jin 已提交
3333
      PERF_TIMER_START(write_pre_and_post_process_time);
3334 3335 3336
      if (updates == &tmp_batch_) {
        tmp_batch_.Clear();
      }
3337
      mutex_.Lock();
3338 3339 3340
      // internal stats
      default_cf_internal_stats_->AddDBStats(
          InternalStats::BYTES_WRITTEN, batch_size);
S
sdong 已提交
3341 3342
      default_cf_internal_stats_->AddDBStats(InternalStats::NUMBER_KEYS_WRITTEN,
                                             my_batch_count);
L
Lei Jin 已提交
3343
      if (!write_options.disableWAL) {
3344 3345 3346 3347 3348
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_SYNCED, 1);
        default_cf_internal_stats_->AddDBStats(
            InternalStats::WAL_FILE_BYTES, log_size);
      }
3349
      if (status.ok()) {
3350
        versions_->SetLastSequence(last_sequence);
3351
      }
J
jorlow@chromium.org 已提交
3352 3353
    }
  }
3354
  if (db_options_.paranoid_checks && !status.ok() &&
3355
      !status.IsTimedOut() && bg_error_.ok()) {
I
Igor Canadi 已提交
3356 3357
    bg_error_ = status; // stop compaction & fail any further writes
  }
3358

I
Igor Canadi 已提交
3359
  write_thread_.ExitWriteThread(&w, last_writer, status);
3360 3361 3362 3363

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

3366
  if (status.IsTimedOut()) {
L
Lei Jin 已提交
3367
    RecordTick(stats_, WRITE_TIMEDOUT);
3368 3369
  }

J
jorlow@chromium.org 已提交
3370 3371 3372
  return status;
}

3373 3374
// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
I
Igor Canadi 已提交
3375
Status DBImpl::DelayWrite(uint64_t expiration_time) {
S
sdong 已提交
3376 3377
  uint64_t time_delayed = 0;
  bool delayed = false;
3378
  bool timed_out = false;
S
sdong 已提交
3379 3380 3381 3382 3383 3384 3385 3386
  {
    StopWatch sw(env_, stats_, WRITE_STALL, &time_delayed);
    bool has_timeout = (expiration_time > 0);
    auto delay = write_controller_.GetDelay();
    if (write_controller_.IsStopped() == false && delay > 0) {
      mutex_.Unlock();
      delayed = true;
      // hopefully we don't have to sleep more than 2 billion microseconds
S
sdong 已提交
3387
      TEST_SYNC_POINT("DBImpl::DelayWrite:Sleep");
S
sdong 已提交
3388 3389 3390
      env_->SleepForMicroseconds(static_cast<int>(delay));
      mutex_.Lock();
    }
3391

S
sdong 已提交
3392 3393 3394
    while (bg_error_.ok() && write_controller_.IsStopped()) {
      delayed = true;
      if (has_timeout) {
S
sdong 已提交
3395
        TEST_SYNC_POINT("DBImpl::DelayWrite:TimedWait");
S
sdong 已提交
3396 3397
        bg_cv_.TimedWait(expiration_time);
        if (env_->NowMicros() > expiration_time) {
3398 3399
          timed_out = true;
          break;
S
sdong 已提交
3400 3401 3402
        }
      } else {
        bg_cv_.Wait();
3403 3404 3405
      }
    }
  }
S
sdong 已提交
3406 3407 3408 3409 3410
  if (delayed) {
    default_cf_internal_stats_->AddDBStats(InternalStats::WRITE_STALL_MICROS,
                                           time_delayed);
    RecordTick(stats_, STALL_MICROS, time_delayed);
  }
I
Igor Canadi 已提交
3411

3412 3413 3414 3415
  if (timed_out) {
    return Status::TimedOut();
  }

I
Igor Canadi 已提交
3416
  return bg_error_;
3417 3418
}

I
Igor Canadi 已提交
3419 3420 3421 3422
Status DBImpl::ScheduleFlushes(WriteContext* context) {
  ColumnFamilyData* cfd;
  while ((cfd = flush_scheduler_.GetNextColumnFamily()) != nullptr) {
    auto status = SetNewMemtableAndNewLogFile(cfd, context);
3423 3424
    SchedulePendingFlush(cfd);
    context->schedule_bg_work_ = true;
I
Igor Canadi 已提交
3425 3426 3427 3428 3429
    if (cfd->Unref()) {
      delete cfd;
    }
    if (!status.ok()) {
      return status;
S
Stanislau Hlebik 已提交
3430 3431
    }
  }
I
Igor Canadi 已提交
3432
  return Status::OK();
S
Stanislau Hlebik 已提交
3433 3434 3435 3436 3437 3438 3439 3440 3441 3442 3443 3444 3445
}

// REQUIRES: mutex_ is held
// REQUIRES: this thread is currently at the front of the writer queue
Status DBImpl::SetNewMemtableAndNewLogFile(ColumnFamilyData* cfd,
                                           WriteContext* context) {
  mutex_.AssertHeld();
  unique_ptr<WritableFile> lfile;
  log::Writer* new_log = nullptr;
  MemTable* new_mem = nullptr;

  // Attempt to switch to a new memtable and trigger flush of old.
  // Do this without holding the dbmutex lock.
3446
  assert(versions_->prev_log_number() == 0);
S
Stanislau Hlebik 已提交
3447 3448 3449 3450
  bool creating_new_log = !log_empty_;
  uint64_t new_log_number =
      creating_new_log ? versions_->NewFileNumber() : logfile_number_;
  SuperVersion* new_superversion = nullptr;
3451
  const MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
S
Stanislau Hlebik 已提交
3452 3453 3454 3455
  mutex_.Unlock();
  Status s;
  {
    if (creating_new_log) {
3456
      s = env_->NewWritableFile(
3457
          LogFileName(db_options_.wal_dir, new_log_number), &lfile,
3458
          env_->OptimizeForLogWrite(env_options_, db_options_));
S
Stanislau Hlebik 已提交
3459 3460 3461
      if (s.ok()) {
        // Our final size should be less than write_buffer_size
        // (compression, etc) but err on the side of caution.
L
Lei Jin 已提交
3462 3463
        lfile->SetPreallocationBlockSize(
            1.1 * mutable_cf_options.write_buffer_size);
S
Stanislau Hlebik 已提交
3464
        new_log = new log::Writer(std::move(lfile));
3465
        log_dir_synced_ = false;
I
Igor Canadi 已提交
3466
      }
S
Stanislau Hlebik 已提交
3467 3468 3469
    }

    if (s.ok()) {
3470
      new_mem = cfd->ConstructNewMemtable(mutable_cf_options);
S
Stanislau Hlebik 已提交
3471 3472 3473
      new_superversion = new SuperVersion();
    }
  }
3474
  Log(InfoLogLevel::DEBUG_LEVEL, db_options_.info_log,
I
Igor Canadi 已提交
3475 3476
      "[%s] New memtable created with log file: #%" PRIu64 "\n",
      cfd->GetName().c_str(), new_log_number);
S
Stanislau Hlebik 已提交
3477 3478 3479 3480 3481 3482 3483 3484 3485 3486 3487
  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);
3488
    logs_to_free_.push_back(log_.release());
S
Stanislau Hlebik 已提交
3489 3490 3491
    log_.reset(new_log);
    log_empty_ = true;
    alive_log_files_.push_back(LogFileNumberSize(logfile_number_));
I
Igor Canadi 已提交
3492
    for (auto loop_cfd : *versions_->GetColumnFamilySet()) {
S
Stanislau Hlebik 已提交
3493 3494 3495 3496
      // 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 已提交
3497 3498 3499
      if (loop_cfd->mem()->GetFirstSequenceNumber() == 0 &&
          loop_cfd->imm()->size() == 0) {
        loop_cfd->SetLogNumber(logfile_number_);
3500
      }
3501 3502
    }
  }
S
Stanislau Hlebik 已提交
3503 3504 3505 3506 3507
  cfd->mem()->SetNextLogNumber(logfile_number_);
  cfd->imm()->Add(cfd->mem());
  new_mem->Ref();
  cfd->SetMemtable(new_mem);
  context->superversions_to_free_.push_back(
3508
      InstallSuperVersion(cfd, new_superversion, mutable_cf_options, true));
3509 3510 3511
  return s;
}

I
Igor Canadi 已提交
3512
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
3513 3514 3515 3516 3517
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

3518 3519
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
3520
  auto version = cfd->current();
3521 3522 3523 3524 3525 3526 3527 3528 3529 3530 3531 3532
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
I
Igor Canadi 已提交
3533
#endif  // ROCKSDB_LITE
3534

I
Igor Canadi 已提交
3535 3536 3537 3538
const std::string& DBImpl::GetName() const {
  return dbname_;
}

3539 3540 3541 3542
Env* DBImpl::GetEnv() const {
  return env_;
}

3543 3544
const Options& DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
I
Igor Canadi 已提交
3545
  return *cfh->cfd()->options();
I
Igor Canadi 已提交
3546 3547
}

3548 3549
const DBOptions& DBImpl::GetDBOptions() const { return db_options_; }

3550
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
3551
                         const Slice& property, std::string* value) {
3552 3553
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3554 3555 3556
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);

3557
  value->clear();
3558 3559 3560 3561 3562
  if (is_int_property) {
    uint64_t int_value;
    bool ret_value = GetIntPropertyInternal(column_family, property_type,
                                            need_out_of_mutex, &int_value);
    if (ret_value) {
3563
      *value = ToString(int_value);
3564 3565 3566 3567 3568
    }
    return ret_value;
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    auto cfd = cfh->cfd();
3569
    InstrumentedMutexLock l(&mutex_);
3570 3571 3572
    return cfd->internal_stats()->GetStringProperty(property_type, property,
                                                    value);
  }
J
jorlow@chromium.org 已提交
3573 3574
}

3575 3576
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
3577 3578
  bool is_int_property = false;
  bool need_out_of_mutex = false;
3579 3580 3581 3582 3583 3584 3585 3586 3587 3588 3589 3590
  DBPropertyType property_type =
      GetPropertyType(property, &is_int_property, &need_out_of_mutex);
  if (!is_int_property) {
    return false;
  }
  return GetIntPropertyInternal(column_family, property_type, need_out_of_mutex,
                                value);
}

bool DBImpl::GetIntPropertyInternal(ColumnFamilyHandle* column_family,
                                    DBPropertyType property_type,
                                    bool need_out_of_mutex, uint64_t* value) {
3591 3592
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
3593 3594

  if (!need_out_of_mutex) {
3595
    InstrumentedMutexLock l(&mutex_);
3596
    return cfd->internal_stats()->GetIntProperty(property_type, value, this);
3597 3598 3599 3600 3601 3602 3603 3604 3605 3606 3607 3608 3609 3610
  } else {
    SuperVersion* sv = GetAndRefSuperVersion(cfd);

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

    ReturnAndCleanupSuperVersion(cfd, sv);

    return ret;
  }
}

SuperVersion* DBImpl::GetAndRefSuperVersion(ColumnFamilyData* cfd) {
  // TODO(ljin): consider using GetReferencedSuperVersion() directly
I
Igor Canadi 已提交
3611
  return cfd->GetThreadLocalSuperVersion(&mutex_);
3612 3613 3614 3615
}

void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
I
Igor Canadi 已提交
3616
  bool unref_sv = !cfd->ReturnThreadLocalSuperVersion(sv);
3617 3618 3619 3620 3621

  if (unref_sv) {
    // Release SuperVersion
    if (sv->Unref()) {
      {
3622
        InstrumentedMutexLock l(&mutex_);
3623 3624 3625 3626 3627 3628 3629
        sv->Cleanup();
      }
      delete sv;
      RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
    }
    RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
  }
3630 3631
}

3632
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
3633
                                 const Range* range, int n, uint64_t* sizes) {
J
jorlow@chromium.org 已提交
3634
  Version* v;
3635 3636
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
J
jorlow@chromium.org 已提交
3637
  {
3638
    InstrumentedMutexLock l(&mutex_);
3639
    v = cfd->current();
3640
    v->Ref();
J
jorlow@chromium.org 已提交
3641 3642 3643 3644
  }

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

  {
3651
    InstrumentedMutexLock l(&mutex_);
J
jorlow@chromium.org 已提交
3652 3653 3654 3655
    v->Unref();
  }
}

I
Igor Canadi 已提交
3656 3657 3658 3659 3660 3661 3662 3663 3664 3665 3666 3667 3668 3669 3670 3671
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 已提交
3672 3673 3674 3675 3676
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
3677
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
3678 3679 3680
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
3681
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
3682 3683
}

3684 3685 3686
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
3687 3688 3689
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
3690 3691
    Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
        "DeleteFile %s failed.\n", name.c_str());
3692 3693 3694
    return Status::InvalidArgument("Invalid file name");
  }

3695 3696 3697 3698
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
3699 3700
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed - not archived log.\n",
3701
          name.c_str());
3702 3703
      return Status::NotSupported("Delete only supported for archived logs");
    }
3704
    status = env_->DeleteFile(db_options_.wal_dir + "/" + name.c_str());
3705
    if (!status.ok()) {
3706 3707
      Log(InfoLogLevel::ERROR_LEVEL, db_options_.info_log,
          "DeleteFile %s failed -- %s.\n",
3708
          name.c_str(), status.ToString().c_str());
3709 3710 3711 3712
    }
    return status;
  }

3713
  int level;
I
Igor Canadi 已提交
3714
  FileMetaData* metadata;
3715
  ColumnFamilyData* cfd;
3716
  VersionEdit edit;
3717
  JobContext job_context(next_job_id_.fetch_add(1), true);
D
Dhruba Borthakur 已提交
3718
  {
3719
    InstrumentedMutexLock l(&mutex_);
3720
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
3721
    if (!status.ok()) {
3722 3723
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
3724
      job_context.Clean();
D
Dhruba Borthakur 已提交
3725 3726
      return Status::InvalidArgument("File not found");
    }
3727
    assert(level < cfd->NumberLevels());
3728

D
Dhruba Borthakur 已提交
3729
    // If the file is being compacted no need to delete.
3730
    if (metadata->being_compacted) {
3731
      Log(InfoLogLevel::INFO_LEVEL, db_options_.info_log,
3732
          "DeleteFile %s Skipped. File about to be compacted\n", name.c_str());
I
Igor Canadi 已提交
3733
      job_context.Clean();
D
Dhruba Borthakur 已提交
3734
      return Status::OK();
3735 3736
    }

D
Dhruba Borthakur 已提交
3737 3738 3739
    // 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 已提交
3740
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
3741
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
3742
      if (vstoreage->NumLevelFiles(i) != 0) {
3743
        Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
3744
            "DeleteFile %s FAILED. File not in last level\n", name.c_str());
I
Igor Canadi 已提交
3745
        job_context.Clean();
D
Dhruba Borthakur 已提交
3746 3747 3748
        return Status::InvalidArgument("File not in last level");
      }
    }
3749
    // if level == 0, it has to be the oldest file
S
sdong 已提交
3750 3751
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
3752 3753
      Log(InfoLogLevel::WARN_LEVEL, db_options_.info_log,
          "DeleteFile %s failed ---"
L
Lei Jin 已提交
3754
          " target file in level 0 must be the oldest.", name.c_str());
I
Igor Canadi 已提交
3755
      job_context.Clean();
3756 3757 3758
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
3759
    edit.DeleteFile(level, number);
3760
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
3761
                                    &edit, &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
3762
    if (status.ok()) {
I
Igor Canadi 已提交
3763 3764
      InstallSuperVersionBackground(cfd, &job_context,
                                    *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
3765
    }
I
Igor Canadi 已提交
3766 3767
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
3768
  LogFlush(db_options_.info_log);
I
Igor Canadi 已提交
3769
  // remove files outside the db-lock
I
Igor Canadi 已提交
3770 3771
  if (job_context.HaveSomethingToDelete()) {
    PurgeObsoleteFiles(job_context);
3772
  }
I
Igor Canadi 已提交
3773
  job_context.Clean();
3774 3775 3776
  return status;
}

3777
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
3778
  InstrumentedMutexLock l(&mutex_);
3779
  versions_->GetLiveFilesMetaData(metadata);
3780
}
3781 3782 3783 3784 3785 3786 3787 3788 3789 3790 3791

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 已提交
3792
#endif  // ROCKSDB_LITE
3793

I
Igor Canadi 已提交
3794 3795 3796 3797 3798 3799 3800
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
3804 3805 3806 3807 3808 3809
    uint64_t fsize = 0;
    Status s = env_->GetFileSize(file_path, &fsize);
    if (!s.ok()) {
      corruption_messages +=
          "Can't access " + md.name + ": " + s.ToString() + "\n";
    } else if (fsize != md.size) {
3810
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
3811
                             ". Size recorded in manifest " +
3812 3813
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
3814 3815 3816 3817 3818 3819 3820 3821 3822
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

3823 3824 3825 3826 3827 3828 3829 3830 3831 3832 3833 3834 3835 3836 3837
Status DBImpl::GetDbIdentity(std::string& identity) {
  std::string idfilename = IdentityFileName(dbname_);
  unique_ptr<SequentialFile> idfile;
  const EnvOptions soptions;
  Status s = env_->NewSequentialFile(idfilename, &idfile, soptions);
  if (!s.ok()) {
    return s;
  }
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
  char buffer[file_size];
  Slice id;
3838
  s = idfile->Read(static_cast<size_t>(file_size), &id, buffer);
3839 3840 3841 3842 3843 3844 3845 3846 3847 3848 3849
  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 已提交
3850 3851
// Default implementations of convenience methods that subclasses of DB
// can call if they wish
3852
Status DB::Put(const WriteOptions& opt, ColumnFamilyHandle* column_family,
3853
               const Slice& key, const Slice& value) {
3854 3855 3856 3857
  // 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);
3858
  batch.Put(column_family, key, value);
J
jorlow@chromium.org 已提交
3859 3860 3861
  return Write(opt, &batch);
}

3862 3863
Status DB::Delete(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                  const Slice& key) {
J
jorlow@chromium.org 已提交
3864
  WriteBatch batch;
3865
  batch.Delete(column_family, key);
J
jorlow@chromium.org 已提交
3866 3867 3868
  return Write(opt, &batch);
}

3869 3870
Status DB::Merge(const WriteOptions& opt, ColumnFamilyHandle* column_family,
                 const Slice& key, const Slice& value) {
3871
  WriteBatch batch;
3872
  batch.Merge(column_family, key, value);
3873 3874 3875
  return Write(opt, &batch);
}

3876
// Default implementation -- returns not supported status
L
Lei Jin 已提交
3877
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
3878
                              const std::string& column_family_name,
3879
                              ColumnFamilyHandle** handle) {
3880
  return Status::NotSupported("");
3881
}
3882
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
3883
  return Status::NotSupported("");
3884 3885
}

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

J
Jim Paton 已提交
3888
Status DB::Open(const Options& options, const std::string& dbname, DB** dbptr) {
3889 3890 3891 3892
  DBOptions db_options(options);
  ColumnFamilyOptions cf_options(options);
  std::vector<ColumnFamilyDescriptor> column_families;
  column_families.push_back(
3893
      ColumnFamilyDescriptor(kDefaultColumnFamilyName, cf_options));
3894
  std::vector<ColumnFamilyHandle*> handles;
3895
  Status s = DB::Open(db_options, dbname, column_families, &handles, dbptr);
3896 3897 3898 3899 3900 3901 3902
  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;
3903 3904
}

3905 3906
Status DB::Open(const DBOptions& db_options, const std::string& dbname,
                const std::vector<ColumnFamilyDescriptor>& column_families,
3907
                std::vector<ColumnFamilyHandle*>* handles, DB** dbptr) {
3908
  Status s = SanitizeOptionsByTable(db_options, column_families);
3909 3910 3911
  if (!s.ok()) {
    return s;
  }
3912

3913
  if (db_options.db_paths.size() > 1) {
3914
    for (auto& cfd : column_families) {
3915 3916
      if ((cfd.options.compaction_style != kCompactionStyleUniversal) &&
          (cfd.options.compaction_style != kCompactionStyleLevel)) {
3917 3918
        return Status::NotSupported(
            "More than one DB paths are only supported in "
3919
            "universal and level compaction styles. ");
3920 3921 3922 3923 3924
      }
    }

    if (db_options.db_paths.size() > 4) {
      return Status::NotSupported(
3925
          "More than four DB paths are not supported yet. ");
3926
    }
3927 3928
  }

3929
  *dbptr = nullptr;
3930
  handles->clear();
J
jorlow@chromium.org 已提交
3931

I
Igor Canadi 已提交
3932 3933 3934 3935
  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);
3936
  }
3937

I
Igor Canadi 已提交
3938
  DBImpl* impl = new DBImpl(db_options, dbname);
3939
  s = impl->env_->CreateDirIfMissing(impl->db_options_.wal_dir);
3940
  if (s.ok()) {
3941
    for (auto db_path : impl->db_options_.db_paths) {
3942
      s = impl->env_->CreateDirIfMissing(db_path.path);
3943 3944 3945 3946 3947 3948
      if (!s.ok()) {
        break;
      }
    }
  }

3949 3950 3951 3952 3953 3954
  if (!s.ok()) {
    delete impl;
    return s;
  }

  s = impl->CreateArchivalDirectory();
3955 3956 3957 3958
  if (!s.ok()) {
    delete impl;
    return s;
  }
J
jorlow@chromium.org 已提交
3959
  impl->mutex_.Lock();
3960 3961
  // Handles create_if_missing, error_if_exists
  s = impl->Recover(column_families);
J
jorlow@chromium.org 已提交
3962
  if (s.ok()) {
3963
    uint64_t new_log_number = impl->versions_->NewFileNumber();
3964
    unique_ptr<WritableFile> lfile;
3965
    EnvOptions soptions(db_options);
3966 3967
    s = impl->db_options_.env->NewWritableFile(
        LogFileName(impl->db_options_.wal_dir, new_log_number), &lfile,
3968 3969
        impl->db_options_.env->OptimizeForLogWrite(soptions,
                                                   impl->db_options_));
J
jorlow@chromium.org 已提交
3970
    if (s.ok()) {
3971
      lfile->SetPreallocationBlockSize(1.1 * max_write_buffer_size);
3972
      impl->logfile_number_ = new_log_number;
3973
      impl->log_.reset(new log::Writer(std::move(lfile)));
I
Igor Canadi 已提交
3974

3975 3976
      // set column family handles
      for (auto cf : column_families) {
I
Igor Canadi 已提交
3977 3978
        auto cfd =
            impl->versions_->GetColumnFamilySet()->GetColumnFamily(cf.name);
I
Igor Canadi 已提交
3979 3980 3981
        if (cfd != nullptr) {
          handles->push_back(
              new ColumnFamilyHandleImpl(cfd, impl, &impl->mutex_));
3982
          impl->NewThreadStatusCfInfo(cfd);
I
Igor Canadi 已提交
3983 3984 3985 3986 3987 3988 3989 3990 3991 3992 3993 3994 3995 3996 3997 3998
        } 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 已提交
3999
        }
4000
      }
I
Igor Canadi 已提交
4001 4002
    }
    if (s.ok()) {
4003
      for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
L
Lei Jin 已提交
4004 4005
        delete impl->InstallSuperVersion(
            cfd, nullptr, *cfd->GetLatestMutableCFOptions());
4006
      }
I
Igor Canadi 已提交
4007 4008
      impl->alive_log_files_.push_back(
          DBImpl::LogFileNumberSize(impl->logfile_number_));
J
jorlow@chromium.org 已提交
4009
      impl->DeleteObsoleteFiles();
4010
      s = impl->directories_.GetDbDir()->Fsync();
J
jorlow@chromium.org 已提交
4011 4012
    }
  }
4013

I
Igor Canadi 已提交
4014 4015
  if (s.ok()) {
    for (auto cfd : *impl->versions_->GetColumnFamilySet()) {
S
sdong 已提交
4016
      if (cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
S
sdong 已提交
4017
        auto* vstorage = cfd->current()->storage_info();
4018
        for (int i = 1; i < vstorage->num_levels(); ++i) {
S
sdong 已提交
4019
          int num_files = vstorage->NumLevelFiles(i);
I
Igor Canadi 已提交
4020
          if (num_files > 0) {
I
Igor Canadi 已提交
4021 4022
            s = Status::InvalidArgument(
                "Not all files are at level 0. Cannot "
S
sdong 已提交
4023
                "open with FIFO compaction style.");
I
Igor Canadi 已提交
4024 4025 4026 4027
            break;
          }
        }
      }
S
sdong 已提交
4028 4029 4030
      if (!cfd->mem()->IsSnapshotSupported()) {
        impl->is_snapshot_supported_ = false;
      }
4031
      if (cfd->ioptions()->merge_operator != nullptr &&
4032 4033 4034 4035 4036
          !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 已提交
4037
      if (!s.ok()) {
4038 4039 4040 4041 4042
        break;
      }
    }
  }

4043 4044
  impl->mutex_.Unlock();

J
jorlow@chromium.org 已提交
4045
  if (s.ok()) {
4046
    impl->opened_successfully_ = true;
S
sdong 已提交
4047 4048
    Log(InfoLogLevel::INFO_LEVEL, impl->db_options_.info_log, "DB pointer %p",
        impl);
J
jorlow@chromium.org 已提交
4049 4050
    *dbptr = impl;
  } else {
Y
Yueh-Hsuan Chiang 已提交
4051
    for (auto* h : *handles) {
4052 4053
      delete h;
    }
4054
    handles->clear();
J
jorlow@chromium.org 已提交
4055 4056 4057 4058 4059
    delete impl;
  }
  return s;
}

4060 4061 4062
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
4063
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
4064 4065
}

4066 4067 4068
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
4069
Status DestroyDB(const std::string& dbname, const Options& options) {
4070
  const InternalKeyComparator comparator(options.comparator);
4071
  const Options& soptions(SanitizeOptions(dbname, &comparator, options));
4072
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
4073
  std::vector<std::string> filenames;
4074

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

J
jorlow@chromium.org 已提交
4078
  FileLock* lock;
4079 4080
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
4081 4082 4083
  if (result.ok()) {
    uint64_t number;
    FileType type;
4084
    InfoLogPrefix info_log_prefix(!options.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
4085
    for (size_t i = 0; i < filenames.size(); i++) {
4086
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
4087
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
4088 4089 4090 4091 4092 4093
        Status del;
        if (type == kMetaDatabase) {
          del = DestroyDB(dbname + "/" + filenames[i], options);
        } else {
          del = env->DeleteFile(dbname + "/" + filenames[i]);
        }
J
jorlow@chromium.org 已提交
4094 4095 4096 4097 4098
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4099

4100 4101
    for (auto& db_path : options.db_paths) {
      env->GetChildren(db_path.path, &filenames);
4102 4103 4104
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
4105
          Status del = env->DeleteFile(db_path.path + "/" + filenames[i]);
4106 4107 4108 4109 4110 4111 4112
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

I
Igor Canadi 已提交
4113 4114 4115 4116 4117 4118 4119 4120 4121 4122 4123 4124 4125 4126 4127 4128 4129 4130
    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;
4131
    env->GetChildren(archivedir, &archiveFiles);
4132 4133
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
4134 4135
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
4136
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
4137 4138 4139 4140 4141
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
4142
    // ignore case where no archival directory is present.
4143
    env->DeleteDir(archivedir);
4144

J
jorlow@chromium.org 已提交
4145
    env->UnlockFile(lock);  // Ignore error since state is already gone
4146
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
4147
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
4148
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
4149 4150 4151 4152
  }
  return result;
}

Y
Yueh-Hsuan Chiang 已提交
4153
#if ROCKSDB_USING_THREAD_STATUS
4154

Y
Yueh-Hsuan Chiang 已提交
4155 4156
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4157
  if (db_options_.enable_thread_tracking) {
4158
    ThreadStatusUtil::NewColumnFamilyInfo(this, cfd);
4159
  }
Y
Yueh-Hsuan Chiang 已提交
4160 4161 4162 4163
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
4164
  if (db_options_.enable_thread_tracking) {
4165
    ThreadStatusUtil::EraseColumnFamilyInfo(cfd);
4166
  }
Y
Yueh-Hsuan Chiang 已提交
4167 4168 4169
}

void DBImpl::EraseThreadStatusDbInfo() const {
4170
  if (db_options_.enable_thread_tracking) {
4171
    ThreadStatusUtil::EraseDatabaseInfo(this);
4172
  }
Y
Yueh-Hsuan Chiang 已提交
4173 4174 4175 4176 4177 4178 4179 4180 4181 4182 4183 4184 4185 4186 4187
}

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

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

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

4188 4189
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
4190
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
4191
#if !defined(IOS_CROSS_COMPILE)
I
Igor Canadi 已提交
4192 4193
  // if we compile with Xcode, we don't run build_detect_vesion, so we don't
  // generate util/build_version.cc
4194 4195
  Log(InfoLogLevel::INFO_LEVEL, log,
      "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR, ROCKSDB_MINOR,
I
Igor Canadi 已提交
4196
      ROCKSDB_PATCH);
4197
  Log(InfoLogLevel::INFO_LEVEL, log, "Git sha %s", rocksdb_build_git_sha);
4198 4199
  Log(InfoLogLevel::INFO_LEVEL, log, "Compile date %s",
      rocksdb_build_compile_date);
I
Igor Canadi 已提交
4200
#endif
4201 4202
}

4203
}  // namespace rocksdb