db_impl.cc 99.4 KB
Newer Older
1
//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
S
Siying Dong 已提交
2 3 4
//  This source code is licensed under both the GPLv2 (found in the
//  COPYING file in the root directory) and Apache 2.0 License
//  (found in the LICENSE.Apache file in the root directory).
5
//
J
jorlow@chromium.org 已提交
6 7 8 9 10
// 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 已提交
11
#ifndef __STDC_FORMAT_MACROS
I
Igor Canadi 已提交
12
#define __STDC_FORMAT_MACROS
L
liuhuahang 已提交
13
#endif
14
#include <stdint.h>
D
David Bernard 已提交
15
#ifdef OS_SOLARIS
D
David Bernard 已提交
16
#include <alloca.h>
D
David Bernard 已提交
17
#endif
18

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

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

98
namespace rocksdb {
99
const std::string kDefaultColumnFamilyName("default");
I
Igor Canadi 已提交
100
void DumpRocksDBBuildVersion(Logger * log);
101

A
Aaron Gao 已提交
102 103 104
CompressionType GetCompressionFlush(
    const ImmutableCFOptions& ioptions,
    const MutableCFOptions& mutable_cf_options) {
105 106 107
  // 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.
108
  if (ioptions.compaction_style == kCompactionStyleUniversal) {
109 110
    if (mutable_cf_options.compaction_options_universal
            .compression_size_percent < 0) {
111 112 113 114 115 116 117
      return mutable_cf_options.compression;
    } else {
      return kNoCompression;
    }
  } else if (!ioptions.compression_per_level.empty()) {
    // For leveled compress when min_level_to_compress != 0.
    return ioptions.compression_per_level[0];
118
  } else {
A
Aaron Gao 已提交
119
    return mutable_cf_options.compression;
120 121
  }
}
I
Igor Canadi 已提交
122

S
Siying Dong 已提交
123
namespace {
124
void DumpSupportInfo(Logger* logger) {
125 126 127 128 129
  ROCKS_LOG_HEADER(logger, "Compression algorithms supported:");
  ROCKS_LOG_HEADER(logger, "\tSnappy supported: %d", Snappy_Supported());
  ROCKS_LOG_HEADER(logger, "\tZlib supported: %d", Zlib_Supported());
  ROCKS_LOG_HEADER(logger, "\tBzip supported: %d", BZip2_Supported());
  ROCKS_LOG_HEADER(logger, "\tLZ4 supported: %d", LZ4_Supported());
130 131
  ROCKS_LOG_HEADER(logger, "\tZSTDNotFinal supported: %d",
                   ZSTDNotFinal_Supported());
132
  ROCKS_LOG_HEADER(logger, "\tZSTD supported: %d", ZSTD_Supported());
133 134
  ROCKS_LOG_HEADER(logger, "Fast CRC32 supported: %s",
                   crc32c::IsFastCrc32Supported().c_str());
I
Igor Canadi 已提交
135
}
136 137

int64_t kDefaultLowPriThrottledRate = 2 * 1024 * 1024;
S
Siying Dong 已提交
138
} // namespace
139

140 141
DBImpl::DBImpl(const DBOptions& options, const std::string& dbname,
               const bool seq_per_batch)
J
jorlow@chromium.org 已提交
142
    : env_(options.env),
H
heyongqiang 已提交
143
      dbname_(dbname),
144 145 146
      initial_db_options_(SanitizeOptions(dbname, options)),
      immutable_db_options_(initial_db_options_),
      mutable_db_options_(initial_db_options_),
147
      stats_(immutable_db_options_.statistics.get()),
148
      db_lock_(nullptr),
149 150
      mutex_(stats_, env_, DB_MUTEX_WAIT_MICROS,
             immutable_db_options_.use_adaptive_mutex),
I
Igor Canadi 已提交
151
      shutting_down_(false),
J
jorlow@chromium.org 已提交
152
      bg_cv_(&mutex_),
153
      logfile_number_(0),
154
      log_dir_synced_(false),
I
Igor Canadi 已提交
155
      log_empty_(true),
156
      default_cf_handle_(nullptr),
157
      log_sync_cv_(&mutex_),
I
Igor Canadi 已提交
158 159
      total_log_size_(0),
      max_total_in_memory_state_(0),
S
sdong 已提交
160
      is_snapshot_supported_(true),
161
      write_buffer_manager_(immutable_db_options_.write_buffer_manager.get()),
162
      write_thread_(immutable_db_options_),
163
      nonmem_write_thread_(immutable_db_options_),
164
      write_controller_(mutable_db_options_.delayed_write_rate),
165 166 167 168 169
      // Use delayed_write_rate as a base line to determine the initial
      // low pri write rate limit. It may be adjusted later.
      low_pri_write_rate_limiter_(NewGenericRateLimiter(std::min(
          static_cast<int64_t>(mutable_db_options_.delayed_write_rate / 8),
          kDefaultLowPriThrottledRate))),
S
sdong 已提交
170
      last_batch_group_size_(0),
171 172
      unscheduled_flushes_(0),
      unscheduled_compactions_(0),
173
      bg_bottom_compaction_scheduled_(0),
174
      bg_compaction_scheduled_(0),
175
      num_running_compactions_(0),
176
      bg_flush_scheduled_(0),
177
      num_running_flushes_(0),
178
      bg_purge_scheduled_(0),
179
      disable_delete_obsolete_files_(0),
180
      delete_obsolete_files_last_run_(env_->NowMicros()),
181
      last_stats_dump_time_microsec_(0),
182
      next_job_id_(1),
183
      has_unpersisted_data_(false),
184
      unable_to_flush_oldest_log_(false),
185
      env_options_(BuildDBOptions(immutable_db_options_, mutable_db_options_)),
186
      env_options_for_compaction_(env_->OptimizeForCompactionTableWrite(
187
          env_options_, immutable_db_options_)),
188
      num_running_ingest_file_(0),
I
Igor Canadi 已提交
189
#ifndef ROCKSDB_LITE
190
      wal_manager_(immutable_db_options_, env_options_, seq_per_batch),
I
Igor Canadi 已提交
191
#endif  // ROCKSDB_LITE
192
      event_logger_(immutable_db_options_.info_log.get()),
193
      bg_work_paused_(0),
194
      bg_compaction_paused_(0),
195
      refitting_level_(false),
196
      opened_successfully_(false),
197
      two_write_queues_(options.two_write_queues),
198
      manual_wal_flush_(options.manual_wal_flush),
199
      seq_per_batch_(seq_per_batch),
200 201 202 203 204 205 206 207 208 209 210
      // last_sequencee_ is always maintained by the main queue that also writes
      // to the memtable. When two_write_queues_ is disabled last seq in
      // memtable is the same as last seq published to the readers. When it is
      // enabled but seq_per_batch_ is disabled, last seq in memtable still
      // indicates last published seq since wal-only writes that go to the 2nd
      // queue do not consume a sequence number. Otherwise writes performed by
      // the 2nd queue could change what is visible to the readers. In this
      // cases, last_seq_same_as_publish_seq_==false, the 2nd queue maintains a
      // separate variable to indicate the last published sequence.
      last_seq_same_as_publish_seq_(
          !(seq_per_batch && options.two_write_queues)),
211 212 213 214
      // Since seq_per_batch_ is currently set only by WritePreparedTxn which
      // requires a custom gc for compaction, we use that to set use_custom_gc_
      // as well.
      use_custom_gc_(seq_per_batch),
215
      preserve_deletes_(options.preserve_deletes) {
H
heyongqiang 已提交
216
  env_->GetAbsolutePath(dbname, &db_absolute_path_);
217

J
jorlow@chromium.org 已提交
218
  // Reserve ten files or so for other uses and give the rest to TableCache.
219
  // Give a large number for setting of "infinite" open files.
L
Leonidas Galanis 已提交
220
  const int table_cache_size = (mutable_db_options_.max_open_files == -1)
221
                                   ? TableCache::kInfiniteCapacity
L
Leonidas Galanis 已提交
222
                                   : mutable_db_options_.max_open_files - 10;
223 224
  table_cache_ = NewLRUCache(table_cache_size,
                             immutable_db_options_.table_cache_numshardbits);
225

226
  versions_.reset(new VersionSet(dbname_, &immutable_db_options_, env_options_,
227
                                 table_cache_.get(), write_buffer_manager_,
228
                                 &write_controller_));
229 230
  column_family_memtables_.reset(
      new ColumnFamilyMemTablesImpl(versions_->GetColumnFamilySet()));
231

232 233 234 235 236
  DumpRocksDBBuildVersion(immutable_db_options_.info_log.get());
  DumpDBFileSummary(immutable_db_options_, dbname_);
  immutable_db_options_.Dump(immutable_db_options_.info_log.get());
  mutable_db_options_.Dump(immutable_db_options_.info_log.get());
  DumpSupportInfo(immutable_db_options_.info_log.get());
237 238 239 240 241

  // always open the DB with 0 here, which means if preserve_deletes_==true
  // we won't drop any deletion markers until SetPreserveDeletesSequenceNumber()
  // is called by client and this seqnum is advanced.
  preserve_deletes_seqnum_.store(0);
J
jorlow@chromium.org 已提交
242 243
}

244
// Will lock the mutex_,  will wait for completion if wait is true
245
void DBImpl::CancelAllBackgroundWork(bool wait) {
246
  InstrumentedMutexLock l(&mutex_);
247

248 249
  ROCKS_LOG_INFO(immutable_db_options_.info_log,
                 "Shutdown: canceling all background work");
250

251
  if (!shutting_down_.load(std::memory_order_acquire) &&
252
      has_unpersisted_data_.load(std::memory_order_relaxed) &&
Y
Yi Wu 已提交
253
      !mutable_db_options_.avoid_flush_during_shutdown) {
254
    for (auto cfd : *versions_->GetColumnFamilySet()) {
255
      if (!cfd->IsDropped() && cfd->initialized() && !cfd->mem()->IsEmpty()) {
256 257 258 259
        cfd->Ref();
        mutex_.Unlock();
        FlushMemTable(cfd, FlushOptions());
        mutex_.Lock();
260
        cfd->Unref();
261 262
      }
    }
263
    versions_->GetColumnFamilySet()->FreeDeadColumnFamilies();
264
  }
265 266 267 268 269 270 271

  shutting_down_.store(true, std::memory_order_release);
  bg_cv_.SignalAll();
  if (!wait) {
    return;
  }
  // Wait for background work to finish
272 273
  while (bg_bottom_compaction_scheduled_ || bg_compaction_scheduled_ ||
         bg_flush_scheduled_) {
274 275 276 277 278
    bg_cv_.Wait();
  }
}

DBImpl::~DBImpl() {
279 280
  // CancelAllBackgroundWork called with false means we just set the shutdown
  // marker. After this we do a variant of the waiting and unschedule work
281 282
  // (to consider: moving all the waiting into CancelAllBackgroundWork(true))
  CancelAllBackgroundWork(false);
283 284
  int bottom_compactions_unscheduled =
      env_->UnSchedule(this, Env::Priority::BOTTOM);
285 286 287
  int compactions_unscheduled = env_->UnSchedule(this, Env::Priority::LOW);
  int flushes_unscheduled = env_->UnSchedule(this, Env::Priority::HIGH);
  mutex_.Lock();
288
  bg_bottom_compaction_scheduled_ -= bottom_compactions_unscheduled;
289 290 291 292
  bg_compaction_scheduled_ -= compactions_unscheduled;
  bg_flush_scheduled_ -= flushes_unscheduled;

  // Wait for background work to finish
293 294
  while (bg_bottom_compaction_scheduled_ || bg_compaction_scheduled_ ||
         bg_flush_scheduled_ || bg_purge_scheduled_) {
295
    TEST_SYNC_POINT("DBImpl::~DBImpl:WaitJob");
296 297
    bg_cv_.Wait();
  }
298
  EraseThreadStatusDbInfo();
I
Igor Canadi 已提交
299 300
  flush_scheduler_.Clear();

301 302 303 304 305 306 307 308 309 310 311 312 313
  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 已提交
314 315 316 317 318
  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();
319 320
  }

I
Igor Canadi 已提交
321 322 323 324 325 326 327 328 329 330
  // 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_) {
331
    JobContext job_context(next_job_id_.fetch_add(1));
I
Igor Canadi 已提交
332
    FindObsoleteFiles(&job_context, true);
333 334

    mutex_.Unlock();
I
Igor Canadi 已提交
335
    // manifest number starting from 2
I
Igor Canadi 已提交
336 337 338
    job_context.manifest_file_number = 1;
    if (job_context.HaveSomethingToDelete()) {
      PurgeObsoleteFiles(job_context);
339
    }
I
Igor Canadi 已提交
340
    job_context.Clean();
341
    mutex_.Lock();
342 343
  }

344 345 346
  for (auto l : logs_to_free_) {
    delete l;
  }
347 348 349
  for (auto& log : logs_) {
    log.ClearWriter();
  }
350
  logs_.clear();
351

352 353 354 355 356 357 358 359 360 361 362 363 364 365 366
  // Table cache may have table handles holding blocks from the block cache.
  // We need to release them before the block cache is destroyed. The block
  // cache may be destroyed inside versions_.reset(), when column family data
  // list is destroyed, so leaving handles in table cache after
  // versions_.reset() may cause issues.
  // Here we clean all unreferenced handles in table cache.
  // Now we assume all user queries have finished, so only version set itself
  // can possibly hold the blocks from block cache. After releasing unreferenced
  // handles here, only handles held by version set left and inside
  // versions_.reset(), we will release them. There, we need to make sure every
  // time a handle is released, we erase it from the cache too. By doing that,
  // we can guarantee that after versions_.reset(), table cache is empty
  // so the cache can be safely destroyed.
  table_cache_->EraseUnRefEntries();

I
Islam AbdelRahman 已提交
367 368 369 370
  for (auto& txn_entry : recovered_transactions_) {
    delete txn_entry.second;
  }

371
  // versions need to be destroyed before table_cache since it can hold
372 373
  // references to table_cache.
  versions_.reset();
374
  mutex_.Unlock();
I
Igor Canadi 已提交
375 376 377
  if (db_lock_ != nullptr) {
    env_->UnlockFile(db_lock_);
  }
378

379
  ROCKS_LOG_INFO(immutable_db_options_.info_log, "Shutdown complete");
380
  LogFlush(immutable_db_options_.info_log);
J
jorlow@chromium.org 已提交
381 382 383
}

void DBImpl::MaybeIgnoreError(Status* s) const {
384
  if (s->ok() || immutable_db_options_.paranoid_checks) {
J
jorlow@chromium.org 已提交
385 386
    // No change needed
  } else {
387 388
    ROCKS_LOG_WARN(immutable_db_options_.info_log, "Ignoring error %s",
                   s->ToString().c_str());
J
jorlow@chromium.org 已提交
389 390 391 392
    *s = Status::OK();
  }
}

393
const Status DBImpl::CreateArchivalDirectory() {
394 395 396
  if (immutable_db_options_.wal_ttl_seconds > 0 ||
      immutable_db_options_.wal_size_limit_mb > 0) {
    std::string archivalPath = ArchivalDirectory(immutable_db_options_.wal_dir);
397 398 399 400 401
    return env_->CreateDirIfMissing(archivalPath);
  }
  return Status::OK();
}

402
void DBImpl::PrintStatistics() {
403
  auto dbstats = immutable_db_options_.statistics.get();
404
  if (dbstats) {
405 406
    ROCKS_LOG_WARN(immutable_db_options_.info_log, "STATISTICS:\n %s",
                   dbstats->ToString().c_str());
407 408 409
  }
}

410
void DBImpl::MaybeDumpStats() {
411 412 413 414 415
  mutex_.Lock();
  unsigned int stats_dump_period_sec =
      mutable_db_options_.stats_dump_period_sec;
  mutex_.Unlock();
  if (stats_dump_period_sec == 0) return;
H
Haobo Xu 已提交
416 417 418

  const uint64_t now_micros = env_->NowMicros();

419
  if (last_stats_dump_time_microsec_ + stats_dump_period_sec * 1000000 <=
420
      now_micros) {
H
Haobo Xu 已提交
421 422 423 424 425
    // 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;
426

427
#ifndef ROCKSDB_LITE
428 429 430 431 432 433 434
    const DBPropertyInfo* cf_property_info =
        GetPropertyInfo(DB::Properties::kCFStats);
    assert(cf_property_info != nullptr);
    const DBPropertyInfo* db_property_info =
        GetPropertyInfo(DB::Properties::kDBStats);
    assert(db_property_info != nullptr);

H
Haobo Xu 已提交
435
    std::string stats;
436
    {
437
      InstrumentedMutexLock l(&mutex_);
438 439
      default_cf_internal_stats_->GetStringProperty(
          *db_property_info, DB::Properties::kDBStats, &stats);
440
      for (auto cfd : *versions_->GetColumnFamilySet()) {
441 442 443 444 445
        if (cfd->initialized()) {
          cfd->internal_stats()->GetStringProperty(
              *cf_property_info, DB::Properties::kCFStatsNoFileHistogram,
              &stats);
        }
446 447
      }
      for (auto cfd : *versions_->GetColumnFamilySet()) {
448 449 450 451
        if (cfd->initialized()) {
          cfd->internal_stats()->GetStringProperty(
              *cf_property_info, DB::Properties::kCFFileHistogram, &stats);
        }
452 453
      }
    }
454 455 456
    ROCKS_LOG_WARN(immutable_db_options_.info_log,
                   "------- DUMPING STATS -------");
    ROCKS_LOG_WARN(immutable_db_options_.info_log, "%s", stats.c_str());
S
Siying Dong 已提交
457 458 459 460
    if (immutable_db_options_.dump_malloc_stats) {
      stats.clear();
      DumpMallocStats(&stats);
      if (!stats.empty()) {
461 462 463
        ROCKS_LOG_WARN(immutable_db_options_.info_log,
                       "------- Malloc STATS -------");
        ROCKS_LOG_WARN(immutable_db_options_.info_log, "%s", stats.c_str());
S
Siying Dong 已提交
464 465
      }
    }
466
#endif  // !ROCKSDB_LITE
467

468
    PrintStatistics();
469 470 471
  }
}

472 473 474 475 476 477 478 479 480 481
void DBImpl::ScheduleBgLogWriterClose(JobContext* job_context) {
  if (!job_context->logs_to_free.empty()) {
    for (auto l : job_context->logs_to_free) {
      AddToLogsToFreeQueue(l);
    }
    job_context->logs_to_free.clear();
    SchedulePurge();
  }
}

S
Siying Dong 已提交
482 483 484 485 486 487
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();
488
  }
S
Siying Dong 已提交
489
  return ret_dir;
490 491
}

S
Siying Dong 已提交
492 493 494 495 496 497 498 499 500 501 502
Status DBImpl::SetOptions(ColumnFamilyHandle* column_family,
    const std::unordered_map<std::string, std::string>& options_map) {
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
  auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (options_map.empty()) {
    ROCKS_LOG_WARN(immutable_db_options_.info_log,
                   "SetOptions() on column family [%s], empty input",
                   cfd->GetName().c_str());
    return Status::InvalidArgument("empty input");
503 504
  }

S
Siying Dong 已提交
505 506 507 508
  MutableCFOptions new_options;
  Status s;
  Status persist_options_status;
  WriteThread::Writer w;
509
  SuperVersionContext sv_context(/* create_superversion */ true);
S
Siying Dong 已提交
510 511 512 513 514 515 516 517 518 519 520 521
  {
    InstrumentedMutexLock l(&mutex_);
    s = cfd->SetOptions(options_map);
    if (s.ok()) {
      new_options = *cfd->GetLatestMutableCFOptions();
      // Append new version to recompute compaction score.
      VersionEdit dummy_edit;
      versions_->LogAndApply(cfd, new_options, &dummy_edit, &mutex_,
                             directories_.GetDbDir());
      // Trigger possible flush/compactions. This has to be before we persist
      // options to file, otherwise there will be a deadlock with writer
      // thread.
522
      InstallSuperVersionAndScheduleWork(cfd, &sv_context, new_options);
523

Y
Yi Wu 已提交
524 525
      persist_options_status = WriteOptionsFile(
          false /*need_mutex_lock*/, true /*need_enter_write_thread*/);
526 527
    }
  }
528
  sv_context.Clean();
529

S
Siying Dong 已提交
530 531 532 533 534 535
  ROCKS_LOG_INFO(immutable_db_options_.info_log,
                 "SetOptions() on column family [%s], inputs:",
                 cfd->GetName().c_str());
  for (const auto& o : options_map) {
    ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s: %s\n", o.first.c_str(),
                   o.second.c_str());
536
  }
S
Siying Dong 已提交
537 538 539 540 541
  if (s.ok()) {
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "[%s] SetOptions() succeeded", cfd->GetName().c_str());
    new_options.Dump(immutable_db_options_.info_log.get());
    if (!persist_options_status.ok()) {
Y
Yi Wu 已提交
542
      s = persist_options_status;
543
    }
544
  } else {
S
Siying Dong 已提交
545 546
    ROCKS_LOG_WARN(immutable_db_options_.info_log, "[%s] SetOptions() failed",
                   cfd->GetName().c_str());
547
  }
S
Siying Dong 已提交
548 549 550
  LogFlush(immutable_db_options_.info_log);
  return s;
#endif  // ROCKSDB_LITE
551 552
}

S
Siying Dong 已提交
553 554 555 556 557 558 559 560 561
Status DBImpl::SetDBOptions(
    const std::unordered_map<std::string, std::string>& options_map) {
#ifdef ROCKSDB_LITE
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
  if (options_map.empty()) {
    ROCKS_LOG_WARN(immutable_db_options_.info_log,
                   "SetDBOptions(), empty input.");
    return Status::InvalidArgument("empty input");
I
Igor Canadi 已提交
562
  }
563

S
Siying Dong 已提交
564 565 566
  MutableDBOptions new_options;
  Status s;
  Status persist_options_status;
567
  bool wal_changed = false;
S
Siying Dong 已提交
568 569 570 571 572 573 574 575 576 577 578 579 580
  WriteThread::Writer w;
  WriteContext write_context;
  {
    InstrumentedMutexLock l(&mutex_);
    s = GetMutableDBOptionsFromStrings(mutable_db_options_, options_map,
                                       &new_options);
    if (s.ok()) {
      if (new_options.max_background_compactions >
          mutable_db_options_.max_background_compactions) {
        env_->IncBackgroundThreadsIfNeeded(
            new_options.max_background_compactions, Env::Priority::LOW);
        MaybeScheduleFlushOrCompaction();
      }
581

582
      write_controller_.set_max_delayed_write_rate(new_options.delayed_write_rate);
L
Leonidas Galanis 已提交
583
      table_cache_.get()->SetCapacity(new_options.max_open_files == -1
584
                                          ? TableCache::kInfiniteCapacity
L
Leonidas Galanis 已提交
585
                                          : new_options.max_open_files - 10);
586 587 588 589 590
      wal_changed = mutable_db_options_.wal_bytes_per_sync !=
                    new_options.wal_bytes_per_sync;
      if (new_options.bytes_per_sync == 0) {
        new_options.bytes_per_sync = 1024 * 1024;
      }
591
      mutable_db_options_ = new_options;
592 593
      env_options_for_compaction_ = EnvOptions(
          BuildDBOptions(immutable_db_options_, mutable_db_options_));
594
      env_options_for_compaction_ = env_->OptimizeForCompactionTableWrite(
595
          env_options_for_compaction_, immutable_db_options_);
596
      versions_->ChangeEnvOptions(mutable_db_options_);
597 598 599 600
      env_options_for_compaction_ = env_->OptimizeForCompactionTableRead(
          env_options_for_compaction_, immutable_db_options_);
      env_options_for_compaction_.compaction_readahead_size =
          mutable_db_options_.compaction_readahead_size;
601
      write_thread_.EnterUnbatched(&w, &mutex_);
602 603
      if (total_log_size_ > GetMaxTotalWalSize() || wal_changed) {
        Status purge_wal_status = SwitchWAL(&write_context);
604 605 606 607 608
        if (!purge_wal_status.ok()) {
          ROCKS_LOG_WARN(immutable_db_options_.info_log,
                         "Unable to purge WAL files in SetDBOptions() -- %s",
                         purge_wal_status.ToString().c_str());
        }
609
      }
Y
Yi Wu 已提交
610 611
      persist_options_status = WriteOptionsFile(
          false /*need_mutex_lock*/, false /*need_enter_write_thread*/);
612
      write_thread_.ExitUnbatched(&w);
613 614
    }
  }
615
  ROCKS_LOG_INFO(immutable_db_options_.info_log, "SetDBOptions(), inputs:");
616
  for (const auto& o : options_map) {
617 618
    ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s: %s\n", o.first.c_str(),
                   o.second.c_str());
619 620
  }
  if (s.ok()) {
621
    ROCKS_LOG_INFO(immutable_db_options_.info_log, "SetDBOptions() succeeded");
622 623 624 625 626 627 628
    new_options.Dump(immutable_db_options_.info_log.get());
    if (!persist_options_status.ok()) {
      if (immutable_db_options_.fail_if_options_file_error) {
        s = Status::IOError(
            "SetDBOptions() succeeded, but unable to persist options",
            persist_options_status.ToString());
      }
629 630 631
      ROCKS_LOG_WARN(immutable_db_options_.info_log,
                     "Unable to persist options in SetDBOptions() -- %s",
                     persist_options_status.ToString().c_str());
632 633
    }
  } else {
634
    ROCKS_LOG_WARN(immutable_db_options_.info_log, "SetDBOptions failed");
L
Lei Jin 已提交
635
  }
636
  LogFlush(immutable_db_options_.info_log);
637
  return s;
I
Igor Canadi 已提交
638
#endif  // ROCKSDB_LITE
639 640
}

641
// return the same level if it cannot be moved
642 643
int DBImpl::FindMinimumEmptyLevelFitting(ColumnFamilyData* cfd,
    const MutableCFOptions& mutable_cf_options, int level) {
644
  mutex_.AssertHeld();
S
sdong 已提交
645
  const auto* vstorage = cfd->current()->storage_info();
646
  int minimum_level = level;
647
  for (int i = level - 1; i > 0; --i) {
648
    // stop if level i is not empty
S
sdong 已提交
649
    if (vstorage->NumLevelFiles(i) > 0) break;
650
    // stop if level i is too small (cannot fit the level files)
651
    if (vstorage->MaxBytesForLevel(i) < vstorage->NumLevelBytes(level)) {
652 653
      break;
    }
654 655 656 657 658 659

    minimum_level = i;
  }
  return minimum_level;
}

660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679
Status DBImpl::FlushWAL(bool sync) {
  {
    // We need to lock log_write_mutex_ since logs_ might change concurrently
    InstrumentedMutexLock wl(&log_write_mutex_);
    log::Writer* cur_log_writer = logs_.back().writer;
    auto s = cur_log_writer->WriteBuffer();
    if (!s.ok()) {
      ROCKS_LOG_ERROR(immutable_db_options_.info_log, "WAL flush error %s",
                      s.ToString().c_str());
    }
    if (!sync) {
      ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "FlushWAL sync=false");
      return s;
    }
  }
  // sync = true
  ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "FlushWAL sync=true");
  return SyncWAL();
}

680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700
Status DBImpl::SyncWAL() {
  autovector<log::Writer*, 1> logs_to_sync;
  bool need_log_dir_sync;
  uint64_t current_log_number;

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

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

    while (logs_.front().number <= current_log_number &&
           logs_.front().getting_synced) {
      log_sync_cv_.Wait();
    }
    // First check that logs are safe to sync in background.
    for (auto it = logs_.begin();
         it != logs_.end() && it->number <= current_log_number; ++it) {
      if (!it->writer->file()->writable_file()->IsSyncThreadSafe()) {
        return Status::NotSupported(
701 702 703 704
            "SyncWAL() is not supported for this implementation of WAL file",
            immutable_db_options_.allow_mmap_writes
                ? "try setting Options::allow_mmap_writes to false"
                : Slice());
705 706 707 708 709
      }
    }
    for (auto it = logs_.begin();
         it != logs_.end() && it->number <= current_log_number; ++it) {
      auto& log = *it;
S
Siying Dong 已提交
710 711 712 713
      assert(!log.getting_synced);
      log.getting_synced = true;
      logs_to_sync.push_back(log.writer);
    }
714

S
Siying Dong 已提交
715
    need_log_dir_sync = !log_dir_synced_;
716 717
  }

718
  TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:1");
S
Siying Dong 已提交
719 720 721 722 723 724 725 726 727 728
  RecordTick(stats_, WAL_FILE_SYNCED);
  Status status;
  for (log::Writer* log : logs_to_sync) {
    status = log->file()->SyncWithoutFlush(immutable_db_options_.use_fsync);
    if (!status.ok()) {
      break;
    }
  }
  if (status.ok() && need_log_dir_sync) {
    status = directories_.GetWalDir()->Fsync();
729
  }
730
  TEST_SYNC_POINT("DBWALTest::SyncWALNotWaitWrite:2");
731

S
Siying Dong 已提交
732 733 734 735 736 737
  TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:1");
  {
    InstrumentedMutexLock l(&mutex_);
    MarkLogsSynced(current_log_number, need_log_dir_sync, status);
  }
  TEST_SYNC_POINT("DBImpl::SyncWAL:BeforeMarkLogsSynced:2");
738

S
Siying Dong 已提交
739
  return status;
740 741
}

S
Siying Dong 已提交
742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763
void DBImpl::MarkLogsSynced(
    uint64_t up_to, bool synced_dir, const Status& status) {
  mutex_.AssertHeld();
  if (synced_dir &&
      logfile_number_ == up_to &&
      status.ok()) {
    log_dir_synced_ = true;
  }
  for (auto it = logs_.begin(); it != logs_.end() && it->number <= up_to;) {
    auto& log = *it;
    assert(log.getting_synced);
    if (status.ok() && logs_.size() > 1) {
      logs_to_free_.push_back(log.ReleaseWriter());
      it = logs_.erase(it);
    } else {
      log.getting_synced = false;
      ++it;
    }
  }
  assert(!status.ok() || logs_.empty() || logs_[0].number > up_to ||
         (logs_.size() == 1 && !logs_[0].getting_synced));
  log_sync_cv_.SignalAll();
764 765
}

S
Siying Dong 已提交
766 767
SequenceNumber DBImpl::GetLatestSequenceNumber() const {
  return versions_->LastSequence();
768 769
}

770 771
void DBImpl::SetLastPublishedSequence(SequenceNumber seq) {
  versions_->SetLastPublishedSequence(seq);
772 773
}

774 775 776 777 778 779 780 781 782
bool DBImpl::SetPreserveDeletesSequenceNumber(SequenceNumber seqnum) {
  if (seqnum > preserve_deletes_seqnum_.load()) {
    preserve_deletes_seqnum_.store(seqnum);
    return true;
  } else {
    return false;
  }
}

S
Siying Dong 已提交
783 784 785 786 787 788 789 790 791
InternalIterator* DBImpl::NewInternalIterator(
    Arena* arena, RangeDelAggregator* range_del_agg,
    ColumnFamilyHandle* column_family) {
  ColumnFamilyData* cfd;
  if (column_family == nullptr) {
    cfd = default_cf_handle_->cfd();
  } else {
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
    cfd = cfh->cfd();
792
  }
793

S
Siying Dong 已提交
794 795 796 797 798 799 800
  mutex_.Lock();
  SuperVersion* super_version = cfd->GetSuperVersion()->Ref();
  mutex_.Unlock();
  ReadOptions roptions;
  return NewInternalIterator(roptions, cfd, super_version, arena,
                             range_del_agg);
}
I
Islam AbdelRahman 已提交
801

S
Siying Dong 已提交
802 803 804
void DBImpl::SchedulePurge() {
  mutex_.AssertHeld();
  assert(opened_successfully_);
I
Islam AbdelRahman 已提交
805

S
Siying Dong 已提交
806 807 808
  // Purge operations are put into High priority queue
  bg_purge_scheduled_++;
  env_->Schedule(&DBImpl::BGWorkPurge, this, Env::Priority::HIGH, nullptr);
I
Islam AbdelRahman 已提交
809 810
}

811 812 813
void DBImpl::BackgroundCallPurge() {
  mutex_.Lock();

814 815 816 817 818 819 820 821 822 823 824 825
  // We use one single loop to clear both queues so that after existing the loop
  // both queues are empty. This is stricter than what is needed, but can make
  // it easier for us to reason the correctness.
  while (!purge_queue_.empty() || !logs_to_free_queue_.empty()) {
    if (!purge_queue_.empty()) {
      auto purge_file = purge_queue_.begin();
      auto fname = purge_file->fname;
      auto type = purge_file->type;
      auto number = purge_file->number;
      auto path_id = purge_file->path_id;
      auto job_id = purge_file->job_id;
      purge_queue_.pop_front();
826

827
      mutex_.Unlock();
S
Siying Dong 已提交
828
      DeleteObsoleteFileImpl(job_id, fname, type, number, path_id);
829 830 831 832 833 834 835 836 837
      mutex_.Lock();
    } else {
      assert(!logs_to_free_queue_.empty());
      log::Writer* log_writer = *(logs_to_free_queue_.begin());
      logs_to_free_queue_.pop_front();
      mutex_.Unlock();
      delete log_writer;
      mutex_.Lock();
    }
838 839 840 841 842 843 844 845 846 847 848
  }
  bg_purge_scheduled_--;

  bg_cv_.SignalAll();
  // 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.
  mutex_.Unlock();
}

849 850
namespace {
struct IterState {
851
  IterState(DBImpl* _db, InstrumentedMutex* _mu, SuperVersion* _super_version,
852
            bool _background_purge)
853 854 855
      : db(_db),
        mu(_mu),
        super_version(_super_version),
856
        background_purge(_background_purge) {}
857 858

  DBImpl* db;
859
  InstrumentedMutex* mu;
860
  SuperVersion* super_version;
861
  bool background_purge;
862 863
};

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

867
  if (state->super_version->Unref()) {
868 869 870
    // Job id == 0 means that this is not our background process, but rather
    // user thread
    JobContext job_context(0);
871

872 873
    state->mu->Lock();
    state->super_version->Cleanup();
I
Igor Canadi 已提交
874
    state->db->FindObsoleteFiles(&job_context, false, true);
875 876 877
    if (state->background_purge) {
      state->db->ScheduleBgLogWriterClose(&job_context);
    }
878 879 880
    state->mu->Unlock();

    delete state->super_version;
I
Igor Canadi 已提交
881
    if (job_context.HaveSomethingToDelete()) {
882
      if (state->background_purge) {
883 884 885 886 887 888 889 890 891 892
        // PurgeObsoleteFiles here does not delete files. Instead, it adds the
        // files to be deleted to a job queue, and deletes it in a separate
        // background thread.
        state->db->PurgeObsoleteFiles(job_context, true /* schedule only */);
        state->mu->Lock();
        state->db->SchedulePurge();
        state->mu->Unlock();
      } else {
        state->db->PurgeObsoleteFiles(job_context);
      }
893
    }
I
Igor Canadi 已提交
894
    job_context.Clean();
I
Igor Canadi 已提交
895
  }
T
Tomislav Novak 已提交
896

897 898
  delete state;
}
H
Hans Wennborg 已提交
899
}  // namespace
900

A
Andrew Kryczka 已提交
901 902 903 904
InternalIterator* DBImpl::NewInternalIterator(
    const ReadOptions& read_options, ColumnFamilyData* cfd,
    SuperVersion* super_version, Arena* arena,
    RangeDelAggregator* range_del_agg) {
S
sdong 已提交
905
  InternalIterator* internal_iter;
906
  assert(arena != nullptr);
A
Andrew Kryczka 已提交
907
  assert(range_del_agg != nullptr);
908
  // Need to create internal iterator from the arena.
909 910 911 912
  MergeIteratorBuilder merge_iter_builder(
      &cfd->internal_comparator(), arena,
      !read_options.total_order_seek &&
          cfd->ioptions()->prefix_extractor != nullptr);
913 914
  // Collect iterator for mutable mem
  merge_iter_builder.AddIterator(
L
Lei Jin 已提交
915
      super_version->mem->NewIterator(read_options, arena));
916
  std::unique_ptr<InternalIterator> range_del_iter;
A
Andrew Kryczka 已提交
917 918
  Status s;
  if (!read_options.ignore_range_deletions) {
919 920
    range_del_iter.reset(
        super_version->mem->NewRangeTombstoneIterator(read_options));
A
Andrew Kryczka 已提交
921 922
    s = range_del_agg->AddTombstones(std::move(range_del_iter));
  }
923
  // Collect all needed child iterators for immutable memtables
A
Andrew Kryczka 已提交
924 925 926 927 928 929 930
  if (s.ok()) {
    super_version->imm->AddIterators(read_options, &merge_iter_builder);
    if (!read_options.ignore_range_deletions) {
      s = super_version->imm->AddRangeTombstoneIterators(read_options, arena,
                                                         range_del_agg);
    }
  }
931
  TEST_SYNC_POINT_CALLBACK("DBImpl::NewInternalIterator:StatusCallback", &s);
A
Andrew Kryczka 已提交
932 933
  if (s.ok()) {
    // Collect iterators for files in L0 - Ln
S
Sagar Vemuri 已提交
934 935 936 937
    if (read_options.read_tier != kMemtableTier) {
      super_version->current->AddIterators(read_options, env_options_,
                                           &merge_iter_builder, range_del_agg);
    }
A
Andrew Kryczka 已提交
938 939 940 941 942 943 944
    internal_iter = merge_iter_builder.Finish();
    IterState* cleanup =
        new IterState(this, &mutex_, super_version,
                      read_options.background_purge_on_iterator_cleanup);
    internal_iter->RegisterCleanup(CleanupIteratorState, cleanup, nullptr);

    return internal_iter;
945 946
  } else {
    CleanupSuperVersion(super_version);
A
Andrew Kryczka 已提交
947
  }
948
  return NewErrorInternalIterator(s, arena);
J
jorlow@chromium.org 已提交
949 950
}

951 952 953 954
ColumnFamilyHandle* DBImpl::DefaultColumnFamily() const {
  return default_cf_handle_;
}

L
Lei Jin 已提交
955
Status DBImpl::Get(const ReadOptions& read_options,
S
Siying Dong 已提交
956 957 958
                   ColumnFamilyHandle* column_family, const Slice& key,
                   PinnableSlice* value) {
  return GetImpl(read_options, column_family, key, value);
L
Lei Jin 已提交
959 960 961
}

Status DBImpl::GetImpl(const ReadOptions& read_options,
962
                       ColumnFamilyHandle* column_family, const Slice& key,
963
                       PinnableSlice* pinnable_val, bool* value_found,
Y
Yi Wu 已提交
964
                       ReadCallback* callback, bool* is_blob_index) {
M
Maysam Yabandeh 已提交
965
  assert(pinnable_val != nullptr);
L
Lei Jin 已提交
966
  StopWatch sw(env_, stats_, DB_GET);
967
  PERF_TIMER_GUARD(get_snapshot_time);
968

969 970 971
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

972 973 974 975 976 977
  // Acquire SuperVersion
  SuperVersion* sv = GetAndRefSuperVersion(cfd);

  TEST_SYNC_POINT("DBImpl::GetImpl:1");
  TEST_SYNC_POINT("DBImpl::GetImpl:2");

978
  SequenceNumber snapshot;
L
Lei Jin 已提交
979
  if (read_options.snapshot != nullptr) {
M
Maysam Yabandeh 已提交
980 981 982 983
    // Note: In WritePrepared txns this is not necessary but not harmful either.
    // Because prep_seq > snapshot => commit_seq > snapshot so if a snapshot is
    // specified we should be fine with skipping seq numbers that are greater
    // than that.
L
Lei Jin 已提交
984 985
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
986
  } else {
987 988 989 990 991 992 993 994 995 996
    // Since we get and reference the super version before getting
    // the snapshot number, without a mutex protection, it is possible
    // that a memtable switch happened in the middle and not all the
    // data for this snapshot is available. But it will contain all
    // the data available in the super version we have, which is also
    // a valid snapshot to read from.
    // We shouldn't get snapshot before finding and referencing the
    // super versipon because a flush happening in between may compact
    // away data for the snapshot, but the snapshot is earlier than the
    // data overwriting it, so users may see wrong results.
997 998 999
    snapshot = last_seq_same_as_publish_seq_
                   ? versions_->LastSequence()
                   : versions_->LastPublishedSequence();
J
jorlow@chromium.org 已提交
1000
  }
1001 1002
  TEST_SYNC_POINT("DBImpl::GetImpl:3");
  TEST_SYNC_POINT("DBImpl::GetImpl:4");
1003

1004
  // Prepare to store a list of merge operations if merge occurs.
1005
  MergeContext merge_context;
1006
  RangeDelAggregator range_del_agg(cfd->internal_comparator(), snapshot);
1007

1008
  Status s;
1009
  // First look in the memtable, then in the immutable memtable (if any).
1010
  // s is both in/out. When in, s could either be OK or MergeInProgress.
1011
  // merge_operands will contain the sequence of merges in the latter case.
1012
  LookupKey lkey(key, snapshot);
L
Lei Jin 已提交
1013
  PERF_TIMER_STOP(get_snapshot_time);
1014

1015 1016
  bool skip_memtable = (read_options.read_tier == kPersistedTier &&
                        has_unpersisted_data_.load(std::memory_order_relaxed));
1017 1018
  bool done = false;
  if (!skip_memtable) {
M
Maysam Yabandeh 已提交
1019
    if (sv->mem->Get(lkey, pinnable_val->GetSelf(), &s, &merge_context,
Y
Yi Wu 已提交
1020
                     &range_del_agg, read_options, callback, is_blob_index)) {
1021
      done = true;
M
Maysam Yabandeh 已提交
1022
      pinnable_val->PinSelf();
1023
      RecordTick(stats_, MEMTABLE_HIT);
A
Andrew Kryczka 已提交
1024
    } else if ((s.ok() || s.IsMergeInProgress()) &&
M
Maysam Yabandeh 已提交
1025
               sv->imm->Get(lkey, pinnable_val->GetSelf(), &s, &merge_context,
Y
Yi Wu 已提交
1026 1027
                            &range_del_agg, read_options, callback,
                            is_blob_index)) {
1028
      done = true;
M
Maysam Yabandeh 已提交
1029
      pinnable_val->PinSelf();
1030 1031
      RecordTick(stats_, MEMTABLE_HIT);
    }
A
Andrew Kryczka 已提交
1032
    if (!done && !s.ok() && !s.IsMergeInProgress()) {
1033
      ReturnAndCleanupSuperVersion(cfd, sv);
A
Andrew Kryczka 已提交
1034 1035
      return s;
    }
1036 1037
  }
  if (!done) {
1038
    PERF_TIMER_GUARD(get_from_output_files_time);
M
Maysam Yabandeh 已提交
1039
    sv->current->Get(read_options, lkey, pinnable_val, &s, &merge_context,
Y
Yi Wu 已提交
1040 1041
                     &range_del_agg, value_found, nullptr, nullptr, callback,
                     is_blob_index);
L
Lei Jin 已提交
1042
    RecordTick(stats_, MEMTABLE_MISS);
1043
  }
1044

1045 1046
  {
    PERF_TIMER_GUARD(get_post_process_time);
1047

1048
    ReturnAndCleanupSuperVersion(cfd, sv);
1049

1050
    RecordTick(stats_, NUMBER_KEYS_READ);
M
Maysam Yabandeh 已提交
1051 1052 1053
    size_t size = pinnable_val->size();
    RecordTick(stats_, BYTES_READ, size);
    MeasureTime(stats_, BYTES_PER_READ, size);
1054
    PERF_COUNTER_ADD(get_read_bytes, size);
1055
  }
1056
  return s;
J
jorlow@chromium.org 已提交
1057 1058
}

1059
std::vector<Status> DBImpl::MultiGet(
L
Lei Jin 已提交
1060
    const ReadOptions& read_options,
1061
    const std::vector<ColumnFamilyHandle*>& column_family,
1062
    const std::vector<Slice>& keys, std::vector<std::string>* values) {
1063

L
Lei Jin 已提交
1064
  StopWatch sw(env_, stats_, DB_MULTIGET);
1065
  PERF_TIMER_GUARD(get_snapshot_time);
K
Kai Liu 已提交
1066

1067
  SequenceNumber snapshot;
1068

1069
  struct MultiGetColumnFamilyData {
I
Igor Canadi 已提交
1070
    ColumnFamilyData* cfd;
1071 1072 1073 1074 1075
    SuperVersion* super_version;
  };
  std::unordered_map<uint32_t, MultiGetColumnFamilyData*> multiget_cf_data;
  // fill up and allocate outside of mutex
  for (auto cf : column_family) {
1076 1077 1078 1079 1080 1081
    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});
1082 1083 1084
    }
  }

1085
  mutex_.Lock();
L
Lei Jin 已提交
1086 1087 1088
  if (read_options.snapshot != nullptr) {
    snapshot = reinterpret_cast<const SnapshotImpl*>(
        read_options.snapshot)->number_;
1089
  } else {
1090 1091 1092
    snapshot = last_seq_same_as_publish_seq_
                   ? versions_->LastSequence()
                   : versions_->LastPublishedSequence();
1093
  }
1094
  for (auto mgd_iter : multiget_cf_data) {
1095 1096
    mgd_iter.second->super_version =
        mgd_iter.second->cfd->GetSuperVersion()->Ref();
1097
  }
1098
  mutex_.Unlock();
1099

1100 1101
  // Contain a list of merge operations if merge occurs.
  MergeContext merge_context;
1102

1103
  // Note: this always resizes the values array
1104 1105 1106
  size_t num_keys = keys.size();
  std::vector<Status> stat_list(num_keys);
  values->resize(num_keys);
1107 1108

  // Keep track of bytes that we read for statistics-recording later
1109
  uint64_t bytes_read = 0;
L
Lei Jin 已提交
1110
  PERF_TIMER_STOP(get_snapshot_time);
1111 1112 1113 1114

  // 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.
1115
  // merge_operands will contain the sequence of merges in the latter case.
1116
  for (size_t i = 0; i < num_keys; ++i) {
1117
    merge_context.Clear();
1118
    Status& s = stat_list[i];
1119 1120 1121
    std::string* value = &(*values)[i];

    LookupKey lkey(keys[i], snapshot);
1122
    auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family[i]);
A
Andrew Kryczka 已提交
1123
    RangeDelAggregator range_del_agg(cfh->cfd()->internal_comparator(),
1124
                                     snapshot);
1125
    auto mgd_iter = multiget_cf_data.find(cfh->cfd()->GetID());
1126 1127 1128
    assert(mgd_iter != multiget_cf_data.end());
    auto mgd = mgd_iter->second;
    auto super_version = mgd->super_version;
1129
    bool skip_memtable =
1130 1131
        (read_options.read_tier == kPersistedTier &&
         has_unpersisted_data_.load(std::memory_order_relaxed));
1132 1133
    bool done = false;
    if (!skip_memtable) {
M
Maysam Yabandeh 已提交
1134
      if (super_version->mem->Get(lkey, value, &s, &merge_context,
A
Andrew Kryczka 已提交
1135
                                  &range_del_agg, read_options)) {
1136 1137
        done = true;
        // TODO(?): RecordTick(stats_, MEMTABLE_HIT)?
M
Maysam Yabandeh 已提交
1138
      } else if (super_version->imm->Get(lkey, value, &s, &merge_context,
A
Andrew Kryczka 已提交
1139
                                         &range_del_agg, read_options)) {
1140 1141 1142 1143 1144
        done = true;
        // TODO(?): RecordTick(stats_, MEMTABLE_HIT)?
      }
    }
    if (!done) {
M
Maysam Yabandeh 已提交
1145
      PinnableSlice pinnable_val;
1146
      PERF_TIMER_GUARD(get_from_output_files_time);
M
Maysam Yabandeh 已提交
1147 1148 1149
      super_version->current->Get(read_options, lkey, &pinnable_val, &s,
                                  &merge_context, &range_del_agg);
      value->assign(pinnable_val.data(), pinnable_val.size());
1150
      // TODO(?): RecordTick(stats_, MEMTABLE_MISS)?
1151 1152 1153
    }

    if (s.ok()) {
M
Maysam Yabandeh 已提交
1154
      bytes_read += value->size();
1155 1156 1157 1158
    }
  }

  // Post processing (decrement reference counts and record statistics)
1159
  PERF_TIMER_GUARD(get_post_process_time);
1160 1161
  autovector<SuperVersion*> superversions_to_delete;

I
Igor Canadi 已提交
1162
  // TODO(icanadi) do we need lock here or just around Cleanup()?
1163 1164 1165 1166 1167 1168
  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);
1169 1170
    }
  }
1171 1172 1173 1174 1175 1176 1177
  mutex_.Unlock();

  for (auto td : superversions_to_delete) {
    delete td;
  }
  for (auto mgd : multiget_cf_data) {
    delete mgd.second;
1178
  }
1179

L
Lei Jin 已提交
1180 1181 1182
  RecordTick(stats_, NUMBER_MULTIGET_CALLS);
  RecordTick(stats_, NUMBER_MULTIGET_KEYS_READ, num_keys);
  RecordTick(stats_, NUMBER_MULTIGET_BYTES_READ, bytes_read);
1183
  MeasureTime(stats_, BYTES_PER_MULTIGET, bytes_read);
1184
  PERF_COUNTER_ADD(multiget_read_bytes, bytes_read);
L
Lei Jin 已提交
1185
  PERF_TIMER_STOP(get_post_process_time);
1186

1187
  return stat_list;
1188 1189
}

L
Lei Jin 已提交
1190
Status DBImpl::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
Y
Yi Wu 已提交
1191
                                  const std::string& column_family,
1192
                                  ColumnFamilyHandle** handle) {
Y
Yi Wu 已提交
1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219 1220 1221 1222 1223 1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234 1235 1236 1237 1238 1239 1240 1241 1242 1243 1244 1245 1246 1247 1248 1249 1250 1251 1252 1253 1254 1255 1256 1257 1258 1259 1260
  assert(handle != nullptr);
  Status s = CreateColumnFamilyImpl(cf_options, column_family, handle);
  if (s.ok()) {
    s = WriteOptionsFile(true /*need_mutex_lock*/,
                         true /*need_enter_write_thread*/);
  }
  return s;
}

Status DBImpl::CreateColumnFamilies(
    const ColumnFamilyOptions& cf_options,
    const std::vector<std::string>& column_family_names,
    std::vector<ColumnFamilyHandle*>* handles) {
  assert(handles != nullptr);
  handles->clear();
  size_t num_cf = column_family_names.size();
  Status s;
  bool success_once = false;
  for (size_t i = 0; i < num_cf; i++) {
    ColumnFamilyHandle* handle;
    s = CreateColumnFamilyImpl(cf_options, column_family_names[i], &handle);
    if (!s.ok()) {
      break;
    }
    handles->push_back(handle);
    success_once = true;
  }
  if (success_once) {
    Status persist_options_status = WriteOptionsFile(
        true /*need_mutex_lock*/, true /*need_enter_write_thread*/);
    if (s.ok() && !persist_options_status.ok()) {
      s = persist_options_status;
    }
  }
  return s;
}

Status DBImpl::CreateColumnFamilies(
    const std::vector<ColumnFamilyDescriptor>& column_families,
    std::vector<ColumnFamilyHandle*>* handles) {
  assert(handles != nullptr);
  handles->clear();
  size_t num_cf = column_families.size();
  Status s;
  bool success_once = false;
  for (size_t i = 0; i < num_cf; i++) {
    ColumnFamilyHandle* handle;
    s = CreateColumnFamilyImpl(column_families[i].options,
                               column_families[i].name, &handle);
    if (!s.ok()) {
      break;
    }
    handles->push_back(handle);
    success_once = true;
  }
  if (success_once) {
    Status persist_options_status = WriteOptionsFile(
        true /*need_mutex_lock*/, true /*need_enter_write_thread*/);
    if (s.ok() && !persist_options_status.ok()) {
      s = persist_options_status;
    }
  }
  return s;
}

Status DBImpl::CreateColumnFamilyImpl(const ColumnFamilyOptions& cf_options,
                                      const std::string& column_family_name,
                                      ColumnFamilyHandle** handle) {
Y
Yueh-Hsuan Chiang 已提交
1261
  Status s;
1262
  Status persist_options_status;
I
Igor Canadi 已提交
1263
  *handle = nullptr;
1264 1265

  s = CheckCompressionSupported(cf_options);
1266
  if (s.ok() && immutable_db_options_.allow_concurrent_memtable_write) {
1267 1268
    s = CheckConcurrentWritesSupported(cf_options);
  }
1269 1270 1271 1272
  if (!s.ok()) {
    return s;
  }

1273
  SuperVersionContext sv_context(/* create_superversion */ true);
Y
Yueh-Hsuan Chiang 已提交
1274
  {
1275
    InstrumentedMutexLock l(&mutex_);
I
Igor Canadi 已提交
1276

Y
Yueh-Hsuan Chiang 已提交
1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289
    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
I
Igor Canadi 已提交
1290
    {  // write thread
1291 1292
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
I
Igor Canadi 已提交
1293 1294
      // LogAndApply will both write the creation in MANIFEST and create
      // ColumnFamilyData object
1295 1296 1297
      s = versions_->LogAndApply(nullptr, MutableCFOptions(cf_options), &edit,
                                 &mutex_, directories_.GetDbDir(), false,
                                 &cf_options);
1298
      write_thread_.ExitUnbatched(&w);
I
Igor Canadi 已提交
1299
    }
Y
Yueh-Hsuan Chiang 已提交
1300 1301 1302 1303 1304
    if (s.ok()) {
      single_column_family_mode_ = false;
      auto* cfd =
          versions_->GetColumnFamilySet()->GetColumnFamily(column_family_name);
      assert(cfd != nullptr);
1305 1306
      InstallSuperVersionAndScheduleWork(
              cfd, &sv_context, *cfd->GetLatestMutableCFOptions());
S
sdong 已提交
1307 1308 1309 1310 1311

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

1312 1313
      cfd->set_initialized();

Y
Yueh-Hsuan Chiang 已提交
1314
      *handle = new ColumnFamilyHandleImpl(cfd, this, &mutex_);
1315 1316 1317
      ROCKS_LOG_INFO(immutable_db_options_.info_log,
                     "Created column family [%s] (ID %u)",
                     column_family_name.c_str(), (unsigned)cfd->GetID());
Y
Yueh-Hsuan Chiang 已提交
1318
    } else {
1319 1320 1321
      ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                      "Creating column family [%s] FAILED -- %s",
                      column_family_name.c_str(), s.ToString().c_str());
Y
Yueh-Hsuan Chiang 已提交
1322
    }
1323
  }  // InstrumentedMutexLock l(&mutex_)
Y
Yueh-Hsuan Chiang 已提交
1324

1325
  sv_context.Clean();
Y
Yueh-Hsuan Chiang 已提交
1326
  // this is outside the mutex
1327
  if (s.ok()) {
1328 1329
    NewThreadStatusCfInfo(
        reinterpret_cast<ColumnFamilyHandleImpl*>(*handle)->cfd());
1330
  }
1331
  return s;
1332 1333
}

1334
Status DBImpl::DropColumnFamily(ColumnFamilyHandle* column_family) {
Y
Yi Wu 已提交
1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348 1349 1350 1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362 1363 1364 1365
  assert(column_family != nullptr);
  Status s = DropColumnFamilyImpl(column_family);
  if (s.ok()) {
    s = WriteOptionsFile(true /*need_mutex_lock*/,
                         true /*need_enter_write_thread*/);
  }
  return s;
}

Status DBImpl::DropColumnFamilies(
    const std::vector<ColumnFamilyHandle*>& column_families) {
  Status s;
  bool success_once = false;
  for (auto* handle : column_families) {
    s = DropColumnFamilyImpl(handle);
    if (!s.ok()) {
      break;
    }
    success_once = true;
  }
  if (success_once) {
    Status persist_options_status = WriteOptionsFile(
        true /*need_mutex_lock*/, true /*need_enter_write_thread*/);
    if (s.ok() && !persist_options_status.ok()) {
      s = persist_options_status;
    }
  }
  return s;
}

Status DBImpl::DropColumnFamilyImpl(ColumnFamilyHandle* column_family) {
1366 1367 1368
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
  if (cfd->GetID() == 0) {
1369 1370
    return Status::InvalidArgument("Can't drop default column family");
  }
1371

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

I
Igor Canadi 已提交
1374 1375
  VersionEdit edit;
  edit.DropColumnFamily();
1376 1377
  edit.SetColumnFamily(cfd->GetID());

1378
  Status s;
1379
  {
1380
    InstrumentedMutexLock l(&mutex_);
1381 1382 1383 1384
    if (cfd->IsDropped()) {
      s = Status::InvalidArgument("Column family already dropped!\n");
    }
    if (s.ok()) {
1385
      // we drop column family from a single write thread
1386 1387
      WriteThread::Writer w;
      write_thread_.EnterUnbatched(&w, &mutex_);
1388 1389
      s = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                 &edit, &mutex_);
1390
      write_thread_.ExitUnbatched(&w);
1391
    }
Y
Yi Wu 已提交
1392 1393 1394 1395 1396
    if (s.ok()) {
      auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
      max_total_in_memory_state_ -= mutable_cf_options->write_buffer_size *
                                    mutable_cf_options->max_write_buffer_number;
    }
S
sdong 已提交
1397 1398 1399 1400 1401 1402

    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()) {
1403
        if (!c->IsDropped() && !c->mem()->IsSnapshotSupported()) {
S
sdong 已提交
1404 1405 1406 1407 1408 1409
          new_is_snapshot_supported = false;
          break;
        }
      }
      is_snapshot_supported_ = new_is_snapshot_supported;
    }
1410
  }
1411

1412
  if (s.ok()) {
Y
Yueh-Hsuan Chiang 已提交
1413 1414 1415 1416
    // 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 已提交
1417
    assert(cfd->IsDropped());
1418 1419
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "Dropped column family with id %u\n", cfd->GetID());
1420
  } else {
1421 1422 1423
    ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                    "Dropping column family with id %u FAILED -- %s\n",
                    cfd->GetID(), s.ToString().c_str());
1424 1425
  }

1426
  return s;
1427 1428
}

L
Lei Jin 已提交
1429
bool DBImpl::KeyMayExist(const ReadOptions& read_options,
1430 1431
                         ColumnFamilyHandle* column_family, const Slice& key,
                         std::string* value, bool* value_found) {
M
Maysam Yabandeh 已提交
1432
  assert(value != nullptr);
1433
  if (value_found != nullptr) {
K
Kai Liu 已提交
1434 1435
    // falsify later if key-may-exist but can't fetch value
    *value_found = true;
1436
  }
L
Lei Jin 已提交
1437
  ReadOptions roptions = read_options;
1438
  roptions.read_tier = kBlockCacheTier; // read from block cache only
M
Maysam Yabandeh 已提交
1439 1440 1441
  PinnableSlice pinnable_val;
  auto s = GetImpl(roptions, column_family, key, &pinnable_val, value_found);
  value->assign(pinnable_val.data(), pinnable_val.size());
K
Kai Liu 已提交
1442

1443
  // If block_cache is enabled and the index block of the table didn't
K
Kai Liu 已提交
1444 1445 1446
  // 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();
1447 1448
}

1449
Iterator* DBImpl::NewIterator(const ReadOptions& read_options,
1450
                              ColumnFamilyHandle* column_family) {
1451
  Iterator* result = nullptr;
1452 1453 1454 1455
  if (read_options.read_tier == kPersistedTier) {
    return NewErrorIterator(Status::NotSupported(
        "ReadTier::kPersistedData is not yet supported in iterators."));
  }
1456 1457 1458 1459 1460 1461 1462 1463 1464
  // if iterator wants internal keys, we can only proceed if
  // we can guarantee the deletes haven't been processed yet
  if (immutable_db_options_.preserve_deletes &&
      read_options.iter_start_seqnum > 0 &&
      read_options.iter_start_seqnum < preserve_deletes_seqnum_.load()) {
        return NewErrorIterator(Status::InvalidArgument(
          "Iterator requested internal keys which are too old and are not"
          " guaranteed to be preserved, try larger iter_start_seqnum opt."));
      }
1465 1466
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
Y
Yi Wu 已提交
1467
  ReadCallback* read_callback = nullptr;  // No read callback provided.
V
Venkatesh Radhakrishnan 已提交
1468 1469 1470
  if (read_options.managed) {
#ifdef ROCKSDB_LITE
    // not supported in lite version
1471
    result =  NewErrorIterator(Status::InvalidArgument(
V
Venkatesh Radhakrishnan 已提交
1472 1473 1474 1475
        "Managed Iterators not supported in RocksDBLite."));
#else
    if ((read_options.tailing) || (read_options.snapshot != nullptr) ||
        (is_snapshot_supported_)) {
1476 1477 1478 1479
      result = new ManagedIterator(this, read_options, cfd);
    } else {
      // Managed iter not supported
      result = NewErrorIterator(Status::InvalidArgument(
V
Venkatesh Radhakrishnan 已提交
1480
        "Managed Iterators not supported without snapshots."));
1481
    }
V
Venkatesh Radhakrishnan 已提交
1482 1483
#endif
  } else if (read_options.tailing) {
I
Igor Canadi 已提交
1484 1485
#ifdef ROCKSDB_LITE
    // not supported in lite version
1486 1487
    result = nullptr;

I
Igor Canadi 已提交
1488
#else
1489 1490
    SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
    auto iter = new ForwardIterator(this, read_options, cfd, sv);
1491
    result = NewDBIterator(
1492
        env_, read_options, *cfd->ioptions(), cfd->user_comparator(), iter,
1493
        kMaxSequenceNumber,
Y
Yi Wu 已提交
1494 1495
        sv->mutable_cf_options.max_sequential_skip_in_iterations,
        read_callback);
I
Igor Canadi 已提交
1496
#endif
T
Tomislav Novak 已提交
1497
  } else {
1498
    // Note: no need to consider the special case of
1499
    // last_seq_same_as_publish_seq_==false since NewIterator is overridden in
1500
    // WritePreparedTxnDB
Y
Yi Wu 已提交
1501 1502 1503
    auto snapshot = read_options.snapshot != nullptr
                        ? read_options.snapshot->GetSequenceNumber()
                        : versions_->LastSequence();
1504
    result = NewIteratorImpl(read_options, cfd, snapshot, read_callback);
1505
  }
1506
  return result;
1507 1508
}

Y
Yi Wu 已提交
1509 1510 1511
ArenaWrappedDBIter* DBImpl::NewIteratorImpl(const ReadOptions& read_options,
                                            ColumnFamilyData* cfd,
                                            SequenceNumber snapshot,
Y
Yi Wu 已提交
1512
                                            ReadCallback* read_callback,
1513 1514
                                            bool allow_blob,
                                            bool allow_refresh) {
Y
Yi Wu 已提交
1515 1516 1517 1518 1519 1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530 1531 1532 1533 1534 1535 1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561
  SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);

  // 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 iterators in
  // the iterator tree are 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(
      env_, read_options, *cfd->ioptions(), snapshot,
      sv->mutable_cf_options.max_sequential_skip_in_iterations,
Y
Yi Wu 已提交
1562
      sv->version_number, read_callback,
1563 1564
      ((read_options.snapshot != nullptr) ? nullptr : this), cfd, allow_blob,
      allow_refresh);
Y
Yi Wu 已提交
1565 1566 1567 1568 1569 1570 1571 1572 1573

  InternalIterator* internal_iter =
      NewInternalIterator(read_options, cfd, sv, db_iter->GetArena(),
                          db_iter->GetRangeDelAggregator());
  db_iter->SetIterUnderDBIter(internal_iter);

  return db_iter;
}

S
Siying Dong 已提交
1574 1575 1576 1577 1578 1579 1580 1581
Status DBImpl::NewIterators(
    const ReadOptions& read_options,
    const std::vector<ColumnFamilyHandle*>& column_families,
    std::vector<Iterator*>* iterators) {
  if (read_options.read_tier == kPersistedTier) {
    return Status::NotSupported(
        "ReadTier::kPersistedData is not yet supported in iterators.");
  }
Y
Yi Wu 已提交
1582
  ReadCallback* read_callback = nullptr;  // No read callback provided.
S
Siying Dong 已提交
1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593
  iterators->clear();
  iterators->reserve(column_families.size());
  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");
1594
    }
S
Siying Dong 已提交
1595 1596 1597 1598
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
      auto iter = new ManagedIterator(this, read_options, cfd);
      iterators->push_back(iter);
1599
    }
S
Siying Dong 已提交
1600 1601 1602 1603 1604 1605 1606 1607 1608 1609 1610
#endif
  } else if (read_options.tailing) {
#ifdef ROCKSDB_LITE
    return Status::InvalidArgument(
        "Tailing interator not supported in RocksDB lite");
#else
    for (auto cfh : column_families) {
      auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(cfh)->cfd();
      SuperVersion* sv = cfd->GetReferencedSuperVersion(&mutex_);
      auto iter = new ForwardIterator(this, read_options, cfd, sv);
      iterators->push_back(NewDBIterator(
1611
          env_, read_options, *cfd->ioptions(), cfd->user_comparator(), iter,
S
Siying Dong 已提交
1612
          kMaxSequenceNumber,
Y
Yi Wu 已提交
1613 1614
          sv->mutable_cf_options.max_sequential_skip_in_iterations,
          read_callback));
1615
    }
S
Siying Dong 已提交
1616 1617
#endif
  } else {
1618
    // Note: no need to consider the special case of
1619
    // last_seq_same_as_publish_seq_==false since NewIterators is overridden in
1620
    // WritePreparedTxnDB
Y
Yi Wu 已提交
1621 1622 1623
    auto snapshot = read_options.snapshot != nullptr
                        ? read_options.snapshot->GetSequenceNumber()
                        : versions_->LastSequence();
S
Siying Dong 已提交
1624 1625 1626
    for (size_t i = 0; i < column_families.size(); ++i) {
      auto* cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(
          column_families[i])->cfd();
Y
Yi Wu 已提交
1627 1628
      iterators->push_back(
          NewIteratorImpl(read_options, cfd, snapshot, read_callback));
1629 1630
    }
  }
1631

I
Igor Canadi 已提交
1632
  return Status::OK();
S
Stanislau Hlebik 已提交
1633 1634
}

S
Siying Dong 已提交
1635
const Snapshot* DBImpl::GetSnapshot() { return GetSnapshotImpl(false); }
1636

S
Siying Dong 已提交
1637 1638 1639
#ifndef ROCKSDB_LITE
const Snapshot* DBImpl::GetSnapshotForWriteConflictBoundary() {
  return GetSnapshotImpl(true);
1640 1641 1642
}
#endif  // ROCKSDB_LITE

S
Siying Dong 已提交
1643 1644 1645 1646 1647 1648 1649 1650 1651 1652
const Snapshot* DBImpl::GetSnapshotImpl(bool is_write_conflict_boundary) {
  int64_t unix_time = 0;
  env_->GetCurrentTime(&unix_time);  // Ignore error
  SnapshotImpl* s = new SnapshotImpl;

  InstrumentedMutexLock l(&mutex_);
  // returns null if the underlying memtable does not support snapshot.
  if (!is_snapshot_supported_) {
    delete s;
    return nullptr;
S
Sage Weil 已提交
1653
  }
1654
  auto snapshot_seq = last_seq_same_as_publish_seq_
1655
                          ? versions_->LastSequence()
1656
                          : versions_->LastPublishedSequence();
1657
  return snapshots_.New(s, snapshot_seq, unix_time, is_write_conflict_boundary);
S
Siying Dong 已提交
1658
}
1659

S
Siying Dong 已提交
1660 1661
void DBImpl::ReleaseSnapshot(const Snapshot* s) {
  const SnapshotImpl* casted_s = reinterpret_cast<const SnapshotImpl*>(s);
S
Stanislau Hlebik 已提交
1662
  {
S
Siying Dong 已提交
1663 1664
    InstrumentedMutexLock l(&mutex_);
    snapshots_.Delete(casted_s);
1665 1666
    uint64_t oldest_snapshot;
    if (snapshots_.empty()) {
1667
      oldest_snapshot = last_seq_same_as_publish_seq_
1668
                            ? versions_->LastSequence()
1669
                            : versions_->LastPublishedSequence();
1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680 1681 1682
    } else {
      oldest_snapshot = snapshots_.oldest()->number_;
    }
    for (auto* cfd : *versions_->GetColumnFamilySet()) {
      cfd->current()->storage_info()->UpdateOldestSnapshot(oldest_snapshot);
      if (!cfd->current()
               ->storage_info()
               ->BottommostFilesMarkedForCompaction()
               .empty()) {
        SchedulePendingCompaction(cfd);
        MaybeScheduleFlushOrCompaction();
      }
    }
1683
  }
S
Siying Dong 已提交
1684
  delete casted_s;
1685 1686
}

I
Igor Canadi 已提交
1687
#ifndef ROCKSDB_LITE
I
Igor Canadi 已提交
1688 1689 1690 1691 1692
Status DBImpl::GetPropertiesOfAllTables(ColumnFamilyHandle* column_family,
                                        TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

1693 1694
  // Increment the ref count
  mutex_.Lock();
I
Igor Canadi 已提交
1695
  auto version = cfd->current();
1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707
  version->Ref();
  mutex_.Unlock();

  auto s = version->GetPropertiesOfAllTables(props);

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

  return s;
}
1708 1709

Status DBImpl::GetPropertiesOfTablesInRange(ColumnFamilyHandle* column_family,
1710
                                            const Range* range, std::size_t n,
1711 1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726 1727 1728 1729 1730
                                            TablePropertiesCollection* props) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

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

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

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

  return s;
}

I
Igor Canadi 已提交
1731
#endif  // ROCKSDB_LITE
1732

I
Igor Canadi 已提交
1733 1734 1735 1736
const std::string& DBImpl::GetName() const {
  return dbname_;
}

1737 1738 1739 1740
Env* DBImpl::GetEnv() const {
  return env_;
}

1741 1742
Options DBImpl::GetOptions(ColumnFamilyHandle* column_family) const {
  InstrumentedMutexLock l(&mutex_);
1743
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1744 1745
  return Options(BuildDBOptions(immutable_db_options_, mutable_db_options_),
                 cfh->cfd()->GetLatestCFOptions());
I
Igor Canadi 已提交
1746 1747
}

1748 1749 1750 1751
DBOptions DBImpl::GetDBOptions() const {
  InstrumentedMutexLock l(&mutex_);
  return BuildDBOptions(immutable_db_options_, mutable_db_options_);
}
1752

1753
bool DBImpl::GetProperty(ColumnFamilyHandle* column_family,
1754
                         const Slice& property, std::string* value) {
1755
  const DBPropertyInfo* property_info = GetPropertyInfo(property);
1756
  value->clear();
1757
  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
1758 1759 1760
  if (property_info == nullptr) {
    return false;
  } else if (property_info->handle_int) {
1761
    uint64_t int_value;
1762 1763
    bool ret_value =
        GetIntPropertyInternal(cfd, *property_info, false, &int_value);
1764
    if (ret_value) {
1765
      *value = ToString(int_value);
1766 1767
    }
    return ret_value;
1768
  } else if (property_info->handle_string) {
1769
    InstrumentedMutexLock l(&mutex_);
1770
    return cfd->internal_stats()->GetStringProperty(*property_info, property,
1771 1772
                                                    value);
  }
1773 1774 1775 1776
  // Shouldn't reach here since exactly one of handle_string and handle_int
  // should be non-nullptr.
  assert(false);
  return false;
J
jorlow@chromium.org 已提交
1777 1778
}

1779 1780
bool DBImpl::GetMapProperty(ColumnFamilyHandle* column_family,
                            const Slice& property,
1781
                            std::map<std::string, std::string>* value) {
1782 1783 1784 1785 1786 1787 1788 1789 1790 1791 1792 1793 1794 1795 1796
  const DBPropertyInfo* property_info = GetPropertyInfo(property);
  value->clear();
  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  if (property_info == nullptr) {
    return false;
  } else if (property_info->handle_map) {
    InstrumentedMutexLock l(&mutex_);
    return cfd->internal_stats()->GetMapProperty(*property_info, property,
                                                 value);
  }
  // If we reach this point it means that handle_map is not provided for the
  // requested property
  return false;
}

1797 1798
bool DBImpl::GetIntProperty(ColumnFamilyHandle* column_family,
                            const Slice& property, uint64_t* value) {
1799 1800
  const DBPropertyInfo* property_info = GetPropertyInfo(property);
  if (property_info == nullptr || property_info->handle_int == nullptr) {
1801 1802
    return false;
  }
1803
  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
1804
  return GetIntPropertyInternal(cfd, *property_info, false, value);
1805 1806
}

1807
bool DBImpl::GetIntPropertyInternal(ColumnFamilyData* cfd,
1808 1809 1810 1811
                                    const DBPropertyInfo& property_info,
                                    bool is_locked, uint64_t* value) {
  assert(property_info.handle_int != nullptr);
  if (!property_info.need_out_of_mutex) {
1812 1813
    if (is_locked) {
      mutex_.AssertHeld();
1814
      return cfd->internal_stats()->GetIntProperty(property_info, value, this);
1815 1816
    } else {
      InstrumentedMutexLock l(&mutex_);
1817
      return cfd->internal_stats()->GetIntProperty(property_info, value, this);
1818
    }
1819
  } else {
1820 1821 1822 1823 1824 1825
    SuperVersion* sv = nullptr;
    if (!is_locked) {
      sv = GetAndRefSuperVersion(cfd);
    } else {
      sv = cfd->GetSuperVersion();
    }
1826 1827

    bool ret = cfd->internal_stats()->GetIntPropertyOutOfMutex(
1828
        property_info, sv->current, value);
1829

1830 1831 1832
    if (!is_locked) {
      ReturnAndCleanupSuperVersion(cfd, sv);
    }
1833 1834 1835 1836 1837

    return ret;
  }
}

S
Siying Dong 已提交
1838 1839 1840 1841
#ifndef ROCKSDB_LITE
Status DBImpl::ResetStats() {
  InstrumentedMutexLock l(&mutex_);
  for (auto* cfd : *versions_->GetColumnFamilySet()) {
1842 1843 1844
    if (cfd->initialized()) {
      cfd->internal_stats()->Clear();
    }
S
Siying Dong 已提交
1845 1846 1847 1848 1849
  }
  return Status::OK();
}
#endif  // ROCKSDB_LITE

1850 1851
bool DBImpl::GetAggregatedIntProperty(const Slice& property,
                                      uint64_t* aggregated_value) {
1852 1853
  const DBPropertyInfo* property_info = GetPropertyInfo(property);
  if (property_info == nullptr || property_info->handle_int == nullptr) {
1854 1855 1856 1857 1858 1859 1860 1861 1862
    return false;
  }

  uint64_t sum = 0;
  {
    // Needs mutex to protect the list of column families.
    InstrumentedMutexLock l(&mutex_);
    uint64_t value;
    for (auto* cfd : *versions_->GetColumnFamilySet()) {
1863 1864 1865
      if (!cfd->initialized()) {
        continue;
      }
1866
      if (GetIntPropertyInternal(cfd, *property_info, true, &value)) {
1867 1868 1869 1870 1871 1872 1873 1874 1875 1876
        sum += value;
      } else {
        return false;
      }
    }
  }
  *aggregated_value = sum;
  return true;
}

1877 1878
SuperVersion* DBImpl::GetAndRefSuperVersion(ColumnFamilyData* cfd) {
  // TODO(ljin): consider using GetReferencedSuperVersion() directly
I
Igor Canadi 已提交
1879
  return cfd->GetThreadLocalSuperVersion(&mutex_);
1880 1881
}

A
agiardullo 已提交
1882 1883 1884 1885 1886 1887 1888 1889 1890 1891 1892 1893
// REQUIRED: this function should only be called on the write thread or if the
// mutex is held.
SuperVersion* DBImpl::GetAndRefSuperVersion(uint32_t column_family_id) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);
  if (!cfd) {
    return nullptr;
  }

  return GetAndRefSuperVersion(cfd);
}

1894 1895 1896 1897 1898 1899 1900 1901 1902 1903 1904 1905 1906
void DBImpl::CleanupSuperVersion(SuperVersion* sv) {
  // Release SuperVersion
  if (sv->Unref()) {
    {
      InstrumentedMutexLock l(&mutex_);
      sv->Cleanup();
    }
    delete sv;
    RecordTick(stats_, NUMBER_SUPERVERSION_CLEANUPS);
  }
  RecordTick(stats_, NUMBER_SUPERVERSION_RELEASES);
}

1907 1908
void DBImpl::ReturnAndCleanupSuperVersion(ColumnFamilyData* cfd,
                                          SuperVersion* sv) {
1909 1910
  if (!cfd->ReturnThreadLocalSuperVersion(sv)) {
    CleanupSuperVersion(sv);
1911
  }
1912 1913
}

A
agiardullo 已提交
1914 1915 1916 1917 1918 1919 1920 1921 1922 1923 1924 1925 1926 1927 1928 1929 1930 1931 1932 1933 1934 1935 1936 1937
// REQUIRED: this function should only be called on the write thread.
void DBImpl::ReturnAndCleanupSuperVersion(uint32_t column_family_id,
                                          SuperVersion* sv) {
  auto column_family_set = versions_->GetColumnFamilySet();
  auto cfd = column_family_set->GetColumnFamily(column_family_id);

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

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

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

  return cf_memtables->GetColumnFamilyHandle();
}

A
Anirban Rahut 已提交
1938 1939 1940 1941 1942 1943 1944 1945 1946 1947 1948 1949 1950 1951
// REQUIRED: mutex is NOT held.
ColumnFamilyHandle* DBImpl::GetColumnFamilyHandleUnlocked(
    uint32_t column_family_id) {
  ColumnFamilyMemTables* cf_memtables = column_family_memtables_.get();

  InstrumentedMutexLock l(&mutex_);

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

  return cf_memtables->GetColumnFamilyHandle();
}

1952 1953 1954 1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967 1968 1969 1970 1971 1972 1973
void DBImpl::GetApproximateMemTableStats(ColumnFamilyHandle* column_family,
                                         const Range& range,
                                         uint64_t* const count,
                                         uint64_t* const size) {
  ColumnFamilyHandleImpl* cfh =
      reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  ColumnFamilyData* cfd = cfh->cfd();
  SuperVersion* sv = GetAndRefSuperVersion(cfd);

  // Convert user_key into a corresponding internal key.
  InternalKey k1(range.start, kMaxSequenceNumber, kValueTypeForSeek);
  InternalKey k2(range.limit, kMaxSequenceNumber, kValueTypeForSeek);
  MemTable::MemTableStats memStats =
      sv->mem->ApproximateStats(k1.Encode(), k2.Encode());
  MemTable::MemTableStats immStats =
      sv->imm->ApproximateStats(k1.Encode(), k2.Encode());
  *count = memStats.count + immStats.count;
  *size = memStats.size + immStats.size;

  ReturnAndCleanupSuperVersion(cfd, sv);
}

1974
void DBImpl::GetApproximateSizes(ColumnFamilyHandle* column_family,
1975
                                 const Range* range, int n, uint64_t* sizes,
1976 1977 1978
                                 uint8_t include_flags) {
  assert(include_flags & DB::SizeApproximationFlags::INCLUDE_FILES ||
         include_flags & DB::SizeApproximationFlags::INCLUDE_MEMTABLES);
J
jorlow@chromium.org 已提交
1979
  Version* v;
1980 1981
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();
1982 1983
  SuperVersion* sv = GetAndRefSuperVersion(cfd);
  v = sv->current;
J
jorlow@chromium.org 已提交
1984 1985 1986

  for (int i = 0; i < n; i++) {
    // Convert user_key into a corresponding internal key.
1987 1988
    InternalKey k1(range[i].start, kMaxSequenceNumber, kValueTypeForSeek);
    InternalKey k2(range[i].limit, kMaxSequenceNumber, kValueTypeForSeek);
V
Vitaliy Liptchinsky 已提交
1989
    sizes[i] = 0;
1990
    if (include_flags & DB::SizeApproximationFlags::INCLUDE_FILES) {
V
Vitaliy Liptchinsky 已提交
1991
      sizes[i] += versions_->ApproximateSize(v, k1.Encode(), k2.Encode());
1992 1993
    }
    if (include_flags & DB::SizeApproximationFlags::INCLUDE_MEMTABLES) {
1994 1995
      sizes[i] += sv->mem->ApproximateStats(k1.Encode(), k2.Encode()).size;
      sizes[i] += sv->imm->ApproximateStats(k1.Encode(), k2.Encode()).size;
1996
    }
J
jorlow@chromium.org 已提交
1997 1998
  }

1999
  ReturnAndCleanupSuperVersion(cfd, sv);
J
jorlow@chromium.org 已提交
2000 2001
}

I
Igor Canadi 已提交
2002 2003 2004 2005 2006 2007 2008 2009 2010 2011 2012 2013 2014 2015 2016 2017
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 已提交
2018 2019 2020 2021 2022
#ifndef ROCKSDB_LITE
Status DBImpl::GetUpdatesSince(
    SequenceNumber seq, unique_ptr<TransactionLogIterator>* iter,
    const TransactionLogIterator::ReadOptions& read_options) {

L
Lei Jin 已提交
2023
  RecordTick(stats_, GET_UPDATES_SINCE_CALLS);
I
Igor Canadi 已提交
2024 2025 2026
  if (seq > versions_->LastSequence()) {
    return Status::NotFound("Requested sequence not yet written in the db");
  }
I
Igor Canadi 已提交
2027
  return wal_manager_.GetUpdatesSince(seq, iter, read_options, versions_.get());
I
Igor Canadi 已提交
2028 2029
}

2030 2031 2032
Status DBImpl::DeleteFile(std::string name) {
  uint64_t number;
  FileType type;
2033 2034 2035
  WalFileType log_type;
  if (!ParseFileName(name, &number, &type, &log_type) ||
      (type != kTableFile && type != kLogFile)) {
2036 2037
    ROCKS_LOG_ERROR(immutable_db_options_.info_log, "DeleteFile %s failed.\n",
                    name.c_str());
2038 2039 2040
    return Status::InvalidArgument("Invalid file name");
  }

2041 2042 2043 2044
  Status status;
  if (type == kLogFile) {
    // Only allow deleting archived log files
    if (log_type != kArchivedLogFile) {
2045 2046 2047
      ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                      "DeleteFile %s failed - not archived log.\n",
                      name.c_str());
2048 2049
      return Status::NotSupported("Delete only supported for archived logs");
    }
C
Changli Gao 已提交
2050
    status = wal_manager_.DeleteFile(name, number);
2051
    if (!status.ok()) {
2052 2053 2054
      ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                      "DeleteFile %s failed -- %s.\n", name.c_str(),
                      status.ToString().c_str());
2055 2056 2057 2058
    }
    return status;
  }

2059
  int level;
I
Igor Canadi 已提交
2060
  FileMetaData* metadata;
2061
  ColumnFamilyData* cfd;
2062
  VersionEdit edit;
2063
  JobContext job_context(next_job_id_.fetch_add(1), true);
D
Dhruba Borthakur 已提交
2064
  {
2065
    InstrumentedMutexLock l(&mutex_);
2066
    status = versions_->GetMetadataForFile(number, &level, &metadata, &cfd);
D
Dhruba Borthakur 已提交
2067
    if (!status.ok()) {
2068 2069
      ROCKS_LOG_WARN(immutable_db_options_.info_log,
                     "DeleteFile %s failed. File not found\n", name.c_str());
I
Igor Canadi 已提交
2070
      job_context.Clean();
D
Dhruba Borthakur 已提交
2071 2072
      return Status::InvalidArgument("File not found");
    }
2073
    assert(level < cfd->NumberLevels());
2074

D
Dhruba Borthakur 已提交
2075
    // If the file is being compacted no need to delete.
2076
    if (metadata->being_compacted) {
2077 2078 2079
      ROCKS_LOG_INFO(immutable_db_options_.info_log,
                     "DeleteFile %s Skipped. File about to be compacted\n",
                     name.c_str());
I
Igor Canadi 已提交
2080
      job_context.Clean();
D
Dhruba Borthakur 已提交
2081
      return Status::OK();
2082 2083
    }

D
Dhruba Borthakur 已提交
2084 2085 2086
    // 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 已提交
2087
    auto* vstoreage = cfd->current()->storage_info();
I
Igor Canadi 已提交
2088
    for (int i = level + 1; i < cfd->NumberLevels(); i++) {
S
sdong 已提交
2089
      if (vstoreage->NumLevelFiles(i) != 0) {
2090 2091 2092
        ROCKS_LOG_WARN(immutable_db_options_.info_log,
                       "DeleteFile %s FAILED. File not in last level\n",
                       name.c_str());
I
Igor Canadi 已提交
2093
        job_context.Clean();
D
Dhruba Borthakur 已提交
2094 2095 2096
        return Status::InvalidArgument("File not in last level");
      }
    }
2097
    // if level == 0, it has to be the oldest file
S
sdong 已提交
2098 2099
    if (level == 0 &&
        vstoreage->LevelFiles(0).back()->fd.GetNumber() != number) {
2100 2101 2102 2103
      ROCKS_LOG_WARN(immutable_db_options_.info_log,
                     "DeleteFile %s failed ---"
                     " target file in level 0 must be the oldest.",
                     name.c_str());
I
Igor Canadi 已提交
2104
      job_context.Clean();
2105 2106 2107
      return Status::InvalidArgument("File in level 0, but not oldest");
    }
    edit.SetColumnFamily(cfd->GetID());
D
Dhruba Borthakur 已提交
2108
    edit.DeleteFile(level, number);
2109
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
2110
                                    &edit, &mutex_, directories_.GetDbDir());
I
Igor Canadi 已提交
2111
    if (status.ok()) {
2112 2113 2114
      InstallSuperVersionAndScheduleWork(
          cfd, &job_context.superversion_context,
          *cfd->GetLatestMutableCFOptions());
I
Igor Canadi 已提交
2115
    }
I
Igor Canadi 已提交
2116 2117
    FindObsoleteFiles(&job_context, false);
  }  // lock released here
2118

2119
  LogFlush(immutable_db_options_.info_log);
2120 2121 2122 2123 2124 2125 2126 2127 2128 2129 2130 2131 2132 2133 2134
  // remove files outside the db-lock
  if (job_context.HaveSomethingToDelete()) {
    // Call PurgeObsoleteFiles() without holding mutex.
    PurgeObsoleteFiles(job_context);
  }
  job_context.Clean();
  return status;
}

Status DBImpl::DeleteFilesInRange(ColumnFamilyHandle* column_family,
                                  const Slice* begin, const Slice* end) {
  Status status;
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  ColumnFamilyData* cfd = cfh->cfd();
  VersionEdit edit;
2135
  std::vector<FileMetaData*> deleted_files;
2136 2137 2138
  JobContext job_context(next_job_id_.fetch_add(1), true);
  {
    InstrumentedMutexLock l(&mutex_);
2139
    Version* input_version = cfd->current();
2140

2141 2142
    auto* vstorage = input_version->storage_info();
    for (int i = 1; i < cfd->NumberLevels(); i++) {
2143 2144 2145 2146 2147 2148 2149 2150 2151
      if (vstorage->LevelFiles(i).empty() ||
          !vstorage->OverlapInLevel(i, begin, end)) {
        continue;
      }
      std::vector<FileMetaData*> level_files;
      InternalKey begin_storage, end_storage, *begin_key, *end_key;
      if (begin == nullptr) {
        begin_key = nullptr;
      } else {
A
Amy Xu 已提交
2152
        begin_storage.SetMinPossibleForUserKey(*begin);
2153 2154 2155 2156 2157
        begin_key = &begin_storage;
      }
      if (end == nullptr) {
        end_key = nullptr;
      } else {
A
Amy Xu 已提交
2158
        end_storage.SetMaxPossibleForUserKey(*end);
2159 2160 2161
        end_key = &end_storage;
      }

2162 2163 2164
      vstorage->GetCleanInputsWithinInterval(i, begin_key, end_key,
                                             &level_files, -1 /* hint_index */,
                                             nullptr /* file_index */);
2165 2166 2167
      FileMetaData* level_file;
      for (uint32_t j = 0; j < level_files.size(); j++) {
        level_file = level_files[j];
2168 2169
        if (level_file->being_compacted) {
          continue;
2170
        }
2171 2172 2173 2174
        edit.SetColumnFamily(cfd->GetID());
        edit.DeleteFile(i, level_file->fd.GetNumber());
        deleted_files.push_back(level_file);
        level_file->being_compacted = true;
2175 2176 2177
      }
    }
    if (edit.GetDeletedFiles().empty()) {
2178
      job_context.Clean();
2179 2180
      return Status::OK();
    }
2181
    input_version->Ref();
2182 2183 2184
    status = versions_->LogAndApply(cfd, *cfd->GetLatestMutableCFOptions(),
                                    &edit, &mutex_, directories_.GetDbDir());
    if (status.ok()) {
2185 2186 2187
      InstallSuperVersionAndScheduleWork(
          cfd, &job_context.superversion_context,
          *cfd->GetLatestMutableCFOptions());
2188
    }
2189 2190 2191 2192
    for (auto* deleted_file : deleted_files) {
      deleted_file->being_compacted = false;
    }
    input_version->Unref();
2193 2194 2195
    FindObsoleteFiles(&job_context, false);
  }  // lock released here

2196
  LogFlush(immutable_db_options_.info_log);
I
Igor Canadi 已提交
2197
  // remove files outside the db-lock
I
Igor Canadi 已提交
2198
  if (job_context.HaveSomethingToDelete()) {
2199
    // Call PurgeObsoleteFiles() without holding mutex.
I
Igor Canadi 已提交
2200
    PurgeObsoleteFiles(job_context);
2201
  }
I
Igor Canadi 已提交
2202
  job_context.Clean();
2203 2204 2205
  return status;
}

2206
void DBImpl::GetLiveFilesMetaData(std::vector<LiveFileMetaData>* metadata) {
2207
  InstrumentedMutexLock l(&mutex_);
2208
  versions_->GetLiveFilesMetaData(metadata);
2209
}
2210 2211 2212 2213 2214 2215 2216 2217 2218 2219 2220

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 已提交
2221
#endif  // ROCKSDB_LITE
2222

I
Igor Canadi 已提交
2223 2224 2225 2226 2227 2228 2229
Status DBImpl::CheckConsistency() {
  mutex_.AssertHeld();
  std::vector<LiveFileMetaData> metadata;
  versions_->GetLiveFilesMetaData(&metadata);

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

I
Igor Canadi 已提交
2233 2234
    uint64_t fsize = 0;
    Status s = env_->GetFileSize(file_path, &fsize);
D
dyniusz 已提交
2235 2236 2237 2238
    if (!s.ok() &&
        env_->GetFileSize(Rocks2LevelTableFileName(file_path), &fsize).ok()) {
      s = Status::OK();
    }
I
Igor Canadi 已提交
2239 2240 2241 2242
    if (!s.ok()) {
      corruption_messages +=
          "Can't access " + md.name + ": " + s.ToString() + "\n";
    } else if (fsize != md.size) {
2243
      corruption_messages += "Sst file size mismatch: " + file_path +
I
Igor Canadi 已提交
2244
                             ". Size recorded in manifest " +
2245 2246
                             ToString(md.size) + ", actual size " +
                             ToString(fsize) + "\n";
I
Igor Canadi 已提交
2247 2248 2249 2250 2251 2252 2253 2254 2255
    }
  }
  if (corruption_messages.size() == 0) {
    return Status::OK();
  } else {
    return Status::Corruption(corruption_messages);
  }
}

2256
Status DBImpl::GetDbIdentity(std::string& identity) const {
2257 2258
  std::string idfilename = IdentityFileName(dbname_);
  const EnvOptions soptions;
2259 2260 2261 2262 2263 2264 2265 2266 2267
  unique_ptr<SequentialFileReader> id_file_reader;
  Status s;
  {
    unique_ptr<SequentialFile> idfile;
    s = env_->NewSequentialFile(idfilename, &idfile, soptions);
    if (!s.ok()) {
      return s;
    }
    id_file_reader.reset(new SequentialFileReader(std::move(idfile)));
2268
  }
2269

2270 2271 2272 2273 2274
  uint64_t file_size;
  s = env_->GetFileSize(idfilename, &file_size);
  if (!s.ok()) {
    return s;
  }
D
Dmitri Smirnov 已提交
2275
  char* buffer = reinterpret_cast<char*>(alloca(file_size));
2276
  Slice id;
2277
  s = id_file_reader->Read(static_cast<size_t>(file_size), &id, buffer);
2278 2279 2280 2281 2282 2283 2284 2285 2286 2287 2288
  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;
}

2289
// Default implementation -- returns not supported status
2290 2291 2292
Status DB::CreateColumnFamily(const ColumnFamilyOptions& cf_options,
                              const std::string& column_family_name,
                              ColumnFamilyHandle** handle) {
2293
  return Status::NotSupported("");
2294
}
Y
Yi Wu 已提交
2295 2296

Status DB::CreateColumnFamilies(
2297 2298 2299
    const ColumnFamilyOptions& cf_options,
    const std::vector<std::string>& column_family_names,
    std::vector<ColumnFamilyHandle*>* handles) {
Y
Yi Wu 已提交
2300 2301 2302 2303
  return Status::NotSupported("");
}

Status DB::CreateColumnFamilies(
2304 2305
    const std::vector<ColumnFamilyDescriptor>& column_families,
    std::vector<ColumnFamilyHandle*>* handles) {
Y
Yi Wu 已提交
2306 2307 2308
  return Status::NotSupported("");
}

2309
Status DB::DropColumnFamily(ColumnFamilyHandle* column_family) {
2310
  return Status::NotSupported("");
2311
}
Y
Yi Wu 已提交
2312 2313

Status DB::DropColumnFamilies(
2314
    const std::vector<ColumnFamilyHandle*>& column_families) {
Y
Yi Wu 已提交
2315 2316 2317
  return Status::NotSupported("");
}

2318 2319 2320 2321
Status DB::DestroyColumnFamilyHandle(ColumnFamilyHandle* column_family) {
  delete column_family;
  return Status::OK();
}
2322

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

2325 2326 2327
Status DB::ListColumnFamilies(const DBOptions& db_options,
                              const std::string& name,
                              std::vector<std::string>* column_families) {
I
Igor Canadi 已提交
2328
  return VersionSet::ListColumnFamilies(column_families, name, db_options.env);
2329 2330
}

2331 2332 2333
Snapshot::~Snapshot() {
}

J
jorlow@chromium.org 已提交
2334
Status DestroyDB(const std::string& dbname, const Options& options) {
2335
  const ImmutableDBOptions soptions(SanitizeOptions(dbname, options));
2336
  Env* env = soptions.env;
J
jorlow@chromium.org 已提交
2337
  std::vector<std::string> filenames;
2338

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

J
jorlow@chromium.org 已提交
2342
  FileLock* lock;
2343 2344
  const std::string lockname = LockFileName(dbname);
  Status result = env->LockFile(lockname, &lock);
J
jorlow@chromium.org 已提交
2345 2346 2347
  if (result.ok()) {
    uint64_t number;
    FileType type;
2348
    InfoLogPrefix info_log_prefix(!soptions.db_log_dir.empty(), dbname);
D
dgrogan@chromium.org 已提交
2349
    for (size_t i = 0; i < filenames.size(); i++) {
2350
      if (ParseFileName(filenames[i], &number, info_log_prefix.prefix, &type) &&
2351
          type != kDBLockFile) {  // Lock file will be deleted at end
K
Kosie van der Merwe 已提交
2352
        Status del;
2353
        std::string path_to_delete = dbname + "/" + filenames[i];
K
Kosie van der Merwe 已提交
2354
        if (type == kMetaDatabase) {
2355 2356
          del = DestroyDB(path_to_delete, options);
        } else if (type == kTableFile) {
2357
          del = DeleteSSTFile(&soptions, path_to_delete, 0);
K
Kosie van der Merwe 已提交
2358
        } else {
2359
          del = env->DeleteFile(path_to_delete);
K
Kosie van der Merwe 已提交
2360
        }
J
jorlow@chromium.org 已提交
2361 2362 2363 2364 2365
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
2366

2367 2368
    for (size_t path_id = 0; path_id < options.db_paths.size(); path_id++) {
      const auto& db_path = options.db_paths[path_id];
2369
      env->GetChildren(db_path.path, &filenames);
2370 2371 2372
      for (size_t i = 0; i < filenames.size(); i++) {
        if (ParseFileName(filenames[i], &number, &type) &&
            type == kTableFile) {  // Lock file will be deleted at end
2373
          std::string table_path = db_path.path + "/" + filenames[i];
2374
          Status del = DeleteSSTFile(&soptions, table_path,
2375
                                     static_cast<uint32_t>(path_id));
2376 2377 2378 2379 2380 2381 2382
          if (result.ok() && !del.ok()) {
            result = del;
          }
        }
      }
    }

I
Igor Canadi 已提交
2383 2384 2385 2386 2387 2388 2389 2390 2391 2392
    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) {
2393
        Status del = env->DeleteFile(LogFileName(soptions.wal_dir, number));
I
Igor Canadi 已提交
2394 2395 2396 2397 2398 2399 2400
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }

    std::vector<std::string> archiveFiles;
2401
    env->GetChildren(archivedir, &archiveFiles);
2402 2403
    // Delete archival files.
    for (size_t i = 0; i < archiveFiles.size(); ++i) {
2404 2405
      if (ParseFileName(archiveFiles[i], &number, &type) &&
          type == kLogFile) {
2406
        Status del = env->DeleteFile(archivedir + "/" + archiveFiles[i]);
2407 2408 2409 2410 2411
        if (result.ok() && !del.ok()) {
          result = del;
        }
      }
    }
2412

I
Islam AbdelRahman 已提交
2413
    // ignore case where no archival directory is present
2414
    env->DeleteDir(archivedir);
2415

J
jorlow@chromium.org 已提交
2416
    env->UnlockFile(lock);  // Ignore error since state is already gone
2417
    env->DeleteFile(lockname);
J
jorlow@chromium.org 已提交
2418
    env->DeleteDir(dbname);  // Ignore error in case dir contains other files
2419
    env->DeleteDir(soptions.wal_dir);
J
jorlow@chromium.org 已提交
2420 2421 2422 2423
  }
  return result;
}

Y
Yi Wu 已提交
2424 2425
Status DBImpl::WriteOptionsFile(bool need_mutex_lock,
                                bool need_enter_write_thread) {
2426
#ifndef ROCKSDB_LITE
Y
Yi Wu 已提交
2427 2428 2429 2430 2431 2432 2433 2434 2435
  WriteThread::Writer w;
  if (need_mutex_lock) {
    mutex_.Lock();
  } else {
    mutex_.AssertHeld();
  }
  if (need_enter_write_thread) {
    write_thread_.EnterUnbatched(&w, &mutex_);
  }
2436 2437 2438

  std::vector<std::string> cf_names;
  std::vector<ColumnFamilyOptions> cf_opts;
2439 2440 2441 2442 2443

  // This part requires mutex to protect the column family options
  for (auto cfd : *versions_->GetColumnFamilySet()) {
    if (cfd->IsDropped()) {
      continue;
2444
    }
2445
    cf_names.push_back(cfd->GetName());
2446
    cf_opts.push_back(cfd->GetLatestCFOptions());
2447 2448
  }

2449 2450
  // Unlock during expensive operations.  New writes cannot get here
  // because the single write thread ensures all new writes get queued.
2451 2452
  DBOptions db_options =
      BuildDBOptions(immutable_db_options_, mutable_db_options_);
2453 2454
  mutex_.Unlock();

2455 2456 2457
  TEST_SYNC_POINT("DBImpl::WriteOptionsFile:1");
  TEST_SYNC_POINT("DBImpl::WriteOptionsFile:2");

2458 2459
  std::string file_name =
      TempOptionsFileName(GetName(), versions_->NewFileNumber());
2460 2461
  Status s =
      PersistRocksDBOptions(db_options, cf_names, cf_opts, file_name, GetEnv());
2462 2463 2464 2465

  if (s.ok()) {
    s = RenameTempFileToOptionsFile(file_name);
  }
Y
Yi Wu 已提交
2466 2467 2468 2469 2470 2471 2472 2473 2474 2475 2476 2477 2478 2479 2480
  // restore lock
  if (!need_mutex_lock) {
    mutex_.Lock();
  }
  if (need_enter_write_thread) {
    write_thread_.ExitUnbatched(&w);
  }
  if (!s.ok()) {
    ROCKS_LOG_WARN(immutable_db_options_.info_log,
                   "Unnable to persist options -- %s", s.ToString().c_str());
    if (immutable_db_options_.fail_if_options_file_error) {
      return Status::IOError("Unable to persist options.",
                             s.ToString().c_str());
    }
  }
2481
#endif  // !ROCKSDB_LITE
Y
Yi Wu 已提交
2482
  return Status::OK();
2483 2484 2485 2486 2487 2488 2489 2490 2491 2492 2493 2494 2495 2496
}

#ifndef ROCKSDB_LITE
namespace {
void DeleteOptionsFilesHelper(const std::map<uint64_t, std::string>& filenames,
                              const size_t num_files_to_keep,
                              const std::shared_ptr<Logger>& info_log,
                              Env* env) {
  if (filenames.size() <= num_files_to_keep) {
    return;
  }
  for (auto iter = std::next(filenames.begin(), num_files_to_keep);
       iter != filenames.end(); ++iter) {
    if (!env->DeleteFile(iter->second).ok()) {
2497 2498
      ROCKS_LOG_WARN(info_log, "Unable to delete options file %s",
                     iter->second.c_str());
2499 2500 2501 2502 2503 2504 2505 2506 2507 2508 2509 2510 2511 2512 2513 2514 2515 2516 2517 2518 2519 2520 2521 2522 2523 2524 2525 2526 2527 2528
    }
  }
}
}  // namespace
#endif  // !ROCKSDB_LITE

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

  // Keeps the latest 2 Options file
  const size_t kNumOptionsFilesKept = 2;
  DeleteOptionsFilesHelper(options_filenames, kNumOptionsFilesKept,
2529
                           immutable_db_options_.info_log, GetEnv());
2530 2531 2532 2533 2534 2535 2536 2537 2538
  return Status::OK();
#else
  return Status::OK();
#endif  // !ROCKSDB_LITE
}

Status DBImpl::RenameTempFileToOptionsFile(const std::string& file_name) {
#ifndef ROCKSDB_LITE
  Status s;
W
Wanning Jiang 已提交
2539 2540

  versions_->options_file_number_ = versions_->NewFileNumber();
2541
  std::string options_file_name =
W
Wanning Jiang 已提交
2542
      OptionsFileName(GetName(), versions_->options_file_number_);
2543 2544 2545 2546 2547 2548 2549 2550 2551 2552
  // Retry if the file name happen to conflict with an existing one.
  s = GetEnv()->RenameFile(file_name, options_file_name);

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

D
Daniel Black 已提交
2553
#ifdef ROCKSDB_USING_THREAD_STATUS
2554

Y
Yueh-Hsuan Chiang 已提交
2555 2556
void DBImpl::NewThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
2557
  if (immutable_db_options_.enable_thread_tracking) {
2558 2559
    ThreadStatusUtil::NewColumnFamilyInfo(this, cfd, cfd->GetName(),
                                          cfd->ioptions()->env);
2560
  }
Y
Yueh-Hsuan Chiang 已提交
2561 2562 2563 2564
}

void DBImpl::EraseThreadStatusCfInfo(
    ColumnFamilyData* cfd) const {
2565
  if (immutable_db_options_.enable_thread_tracking) {
2566
    ThreadStatusUtil::EraseColumnFamilyInfo(cfd);
2567
  }
Y
Yueh-Hsuan Chiang 已提交
2568 2569 2570
}

void DBImpl::EraseThreadStatusDbInfo() const {
2571
  if (immutable_db_options_.enable_thread_tracking) {
2572
    ThreadStatusUtil::EraseDatabaseInfo(this);
2573
  }
Y
Yueh-Hsuan Chiang 已提交
2574 2575 2576 2577 2578 2579 2580 2581 2582 2583 2584 2585 2586 2587 2588
}

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

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

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

2589 2590
//
// A global method that can dump out the build version
I
Igor Canadi 已提交
2591
void DumpRocksDBBuildVersion(Logger * log) {
I
Igor Canadi 已提交
2592
#if !defined(IOS_CROSS_COMPILE)
H
hyunwoo 已提交
2593
  // if we compile with Xcode, we don't run build_detect_version, so we don't
I
Igor Canadi 已提交
2594
  // generate util/build_version.cc
2595 2596 2597 2598
  ROCKS_LOG_HEADER(log, "RocksDB version: %d.%d.%d\n", ROCKSDB_MAJOR,
                   ROCKSDB_MINOR, ROCKSDB_PATCH);
  ROCKS_LOG_HEADER(log, "Git sha %s", rocksdb_build_git_sha);
  ROCKS_LOG_HEADER(log, "Compile date %s", rocksdb_build_compile_date);
I
Igor Canadi 已提交
2599
#endif
2600 2601
}

A
agiardullo 已提交
2602 2603 2604 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618
#ifndef ROCKSDB_LITE
SequenceNumber DBImpl::GetEarliestMemTableSequenceNumber(SuperVersion* sv,
                                                         bool include_history) {
  // Find the earliest sequence number that we know we can rely on reading
  // from the memtable without needing to check sst files.
  SequenceNumber earliest_seq =
      sv->imm->GetEarliestSequenceNumber(include_history);
  if (earliest_seq == kMaxSequenceNumber) {
    earliest_seq = sv->mem->GetEarliestSequenceNumber();
  }
  assert(sv->mem->GetEarliestSequenceNumber() >= earliest_seq);

  return earliest_seq;
}
#endif  // ROCKSDB_LITE

#ifndef ROCKSDB_LITE
2619 2620
Status DBImpl::GetLatestSequenceForKey(SuperVersion* sv, const Slice& key,
                                       bool cache_only, SequenceNumber* seq,
2621 2622
                                       bool* found_record_for_key,
                                       bool* is_blob_index) {
A
agiardullo 已提交
2623 2624
  Status s;
  MergeContext merge_context;
2625 2626
  RangeDelAggregator range_del_agg(sv->mem->GetInternalKeyComparator(),
                                   kMaxSequenceNumber);
A
agiardullo 已提交
2627

A
Andrew Kryczka 已提交
2628
  ReadOptions read_options;
A
agiardullo 已提交
2629 2630 2631 2632
  SequenceNumber current_seq = versions_->LastSequence();
  LookupKey lkey(key, current_seq);

  *seq = kMaxSequenceNumber;
2633 2634
  *found_record_for_key = false;

A
agiardullo 已提交
2635
  // Check if there is a record for this key in the latest memtable
M
Maysam Yabandeh 已提交
2636
  sv->mem->Get(lkey, nullptr, &s, &merge_context, &range_del_agg, seq,
2637
               read_options, nullptr /*read_callback*/, is_blob_index);
A
agiardullo 已提交
2638 2639 2640

  if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
    // unexpected error reading memtable.
2641 2642 2643
    ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                    "Unexpected status returned from MemTable::Get: %s\n",
                    s.ToString().c_str());
A
agiardullo 已提交
2644 2645 2646 2647 2648 2649

    return s;
  }

  if (*seq != kMaxSequenceNumber) {
    // Found a sequence number, no need to check immutable memtables
2650
    *found_record_for_key = true;
A
agiardullo 已提交
2651 2652 2653 2654
    return Status::OK();
  }

  // Check if there is a record for this key in the immutable memtables
M
Maysam Yabandeh 已提交
2655
  sv->imm->Get(lkey, nullptr, &s, &merge_context, &range_del_agg, seq,
2656
               read_options, nullptr /*read_callback*/, is_blob_index);
A
agiardullo 已提交
2657 2658 2659

  if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
    // unexpected error reading memtable.
2660 2661 2662
    ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                    "Unexpected status returned from MemTableList::Get: %s\n",
                    s.ToString().c_str());
A
agiardullo 已提交
2663 2664 2665 2666 2667 2668

    return s;
  }

  if (*seq != kMaxSequenceNumber) {
    // Found a sequence number, no need to check memtable history
2669
    *found_record_for_key = true;
A
agiardullo 已提交
2670 2671 2672 2673
    return Status::OK();
  }

  // Check if there is a record for this key in the immutable memtables
A
Andrew Kryczka 已提交
2674
  sv->imm->GetFromHistory(lkey, nullptr, &s, &merge_context, &range_del_agg,
2675
                          seq, read_options, is_blob_index);
A
agiardullo 已提交
2676 2677 2678

  if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
    // unexpected error reading memtable.
2679 2680
    ROCKS_LOG_ERROR(
        immutable_db_options_.info_log,
A
agiardullo 已提交
2681 2682 2683 2684 2685 2686
        "Unexpected status returned from MemTableList::GetFromHistory: %s\n",
        s.ToString().c_str());

    return s;
  }

2687 2688 2689 2690 2691 2692 2693 2694 2695 2696
  if (*seq != kMaxSequenceNumber) {
    // Found a sequence number, no need to check SST files
    *found_record_for_key = true;
    return Status::OK();
  }

  // TODO(agiardullo): possible optimization: consider checking cached
  // SST files if cache_only=true?
  if (!cache_only) {
    // Check tables
R
Reid Horuff 已提交
2697
    sv->current->Get(read_options, lkey, nullptr, &s, &merge_context,
A
Andrew Kryczka 已提交
2698
                     &range_del_agg, nullptr /* value_found */,
2699 2700
                     found_record_for_key, seq, nullptr /*read_callback*/,
                     is_blob_index);
2701 2702 2703

    if (!(s.ok() || s.IsNotFound() || s.IsMergeInProgress())) {
      // unexpected error reading SST files
2704 2705 2706
      ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                      "Unexpected status returned from Version::Get: %s\n",
                      s.ToString().c_str());
2707 2708 2709 2710 2711

      return s;
    }
  }

A
agiardullo 已提交
2712 2713
  return Status::OK();
}
2714 2715 2716 2717 2718 2719 2720 2721 2722

Status DBImpl::IngestExternalFile(
    ColumnFamilyHandle* column_family,
    const std::vector<std::string>& external_files,
    const IngestExternalFileOptions& ingestion_options) {
  Status status;
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

2723 2724 2725 2726 2727 2728 2729 2730 2731
  // Ingest should immediately fail if ingest_behind is requested,
  // but the DB doesn't support it.
  if (ingestion_options.ingest_behind) {
    if (!immutable_db_options_.allow_ingest_behind) {
      return Status::InvalidArgument(
        "Can't ingest_behind file in DB with allow_ingest_behind=false");
    }
  }

2732 2733 2734 2735 2736 2737 2738
  ExternalSstFileIngestionJob ingestion_job(env_, versions_.get(), cfd,
                                            immutable_db_options_, env_options_,
                                            &snapshots_, ingestion_options);

  std::list<uint64_t>::iterator pending_output_elem;
  {
    InstrumentedMutexLock l(&mutex_);
2739 2740 2741 2742 2743 2744
    if (!bg_error_.ok()) {
      // Don't ingest files when there is a bg_error
      return bg_error_;
    }

    // Make sure that bg cleanup wont delete the files that we are ingesting
2745 2746 2747 2748 2749 2750 2751 2752
    pending_output_elem = CaptureCurrentFileNumberInPendingOutputs();
  }

  status = ingestion_job.Prepare(external_files);
  if (!status.ok()) {
    return status;
  }

2753
  SuperVersionContext sv_context(/* create_superversion */ true);
2754
  TEST_SYNC_POINT("DBImpl::AddFile:Start");
2755 2756 2757 2758 2759
  {
    // Lock db mutex
    InstrumentedMutexLock l(&mutex_);
    TEST_SYNC_POINT("DBImpl::AddFile:MutexLock");

2760
    // Stop writes to the DB by entering both write threads
2761 2762
    WriteThread::Writer w;
    write_thread_.EnterUnbatched(&w, &mutex_);
2763
    WriteThread::Writer nonmem_w;
2764
    if (two_write_queues_) {
2765 2766
      nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_);
    }
2767

2768 2769
    num_running_ingest_file_++;

2770 2771 2772 2773 2774 2775
    // We cannot ingest a file into a dropped CF
    if (cfd->IsDropped()) {
      status = Status::InvalidArgument(
          "Cannot ingest an external file into a dropped CF");
    }

2776
    // Figure out if we need to flush the memtable first
2777 2778 2779
    if (status.ok()) {
      bool need_flush = false;
      status = ingestion_job.NeedsFlush(&need_flush);
2780 2781
      TEST_SYNC_POINT_CALLBACK("DBImpl::IngestExternalFile:NeedFlush",
                               &need_flush);
2782 2783 2784 2785 2786
      if (status.ok() && need_flush) {
        mutex_.Unlock();
        status = FlushMemTable(cfd, FlushOptions(), true /* writes_stopped */);
        mutex_.Lock();
      }
2787 2788 2789 2790 2791 2792 2793 2794 2795 2796 2797 2798 2799 2800 2801
    }

    // Run the ingestion job
    if (status.ok()) {
      status = ingestion_job.Run();
    }

    // Install job edit [Mutex will be unlocked here]
    auto mutable_cf_options = cfd->GetLatestMutableCFOptions();
    if (status.ok()) {
      status =
          versions_->LogAndApply(cfd, *mutable_cf_options, ingestion_job.edit(),
                                 &mutex_, directories_.GetDbDir());
    }
    if (status.ok()) {
2802 2803
      InstallSuperVersionAndScheduleWork(cfd, &sv_context,
                                         *mutable_cf_options);
2804 2805 2806
    }

    // Resume writes to the DB
2807
    if (two_write_queues_) {
2808 2809
      nonmem_write_thread_.ExitUnbatched(&nonmem_w);
    }
2810 2811 2812 2813 2814 2815 2816 2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828
    write_thread_.ExitUnbatched(&w);

    // Update stats
    if (status.ok()) {
      ingestion_job.UpdateStats();
    }

    ReleaseFileNumberFromPendingOutputs(pending_output_elem);

    num_running_ingest_file_--;
    if (num_running_ingest_file_ == 0) {
      bg_cv_.SignalAll();
    }

    TEST_SYNC_POINT("DBImpl::AddFile:MutexUnlock");
  }
  // mutex_ is unlocked here

  // Cleanup
2829
  sv_context.Clean();
2830 2831
  ingestion_job.Cleanup(status);

2832 2833 2834 2835
  if (status.ok()) {
    NotifyOnExternalFileIngested(cfd, ingestion_job);
  }

2836 2837 2838
  return status;
}

A
Aaron G 已提交
2839 2840 2841 2842 2843 2844 2845 2846 2847 2848 2849 2850 2851 2852 2853 2854 2855 2856 2857 2858 2859 2860 2861 2862 2863 2864
Status DBImpl::VerifyChecksum() {
  Status s;
  Options options;
  EnvOptions env_options;
  std::vector<ColumnFamilyData*> cfd_list;
  {
    InstrumentedMutexLock l(&mutex_);
    for (auto cfd : *versions_->GetColumnFamilySet()) {
      if (!cfd->IsDropped() && cfd->initialized()) {
        cfd->Ref();
        cfd_list.push_back(cfd);
      }
    }
  }
  std::vector<SuperVersion*> sv_list;
  for (auto cfd : cfd_list) {
    sv_list.push_back(cfd->GetReferencedSuperVersion(&mutex_));
  }
  for (auto& sv : sv_list) {
    VersionStorageInfo* vstorage = sv->current->storage_info();
    for (int i = 0; i < vstorage->num_non_empty_levels() && s.ok(); i++) {
      for (size_t j = 0; j < vstorage->LevelFilesBrief(i).num_files && s.ok();
           j++) {
        const auto& fd = vstorage->LevelFilesBrief(i).files[j].fd;
        std::string fname = TableFileName(immutable_db_options_.db_paths,
                                          fd.GetNumber(), fd.GetPathId());
2865
        s = rocksdb::VerifySstFileChecksum(options, env_options, fname);
A
Aaron G 已提交
2866 2867 2868 2869 2870 2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886
      }
    }
    if (!s.ok()) {
      break;
    }
  }
  {
    InstrumentedMutexLock l(&mutex_);
    for (auto sv : sv_list) {
      if (sv && sv->Unref()) {
        sv->Cleanup();
        delete sv;
      }
    }
    for (auto cfd : cfd_list) {
        cfd->Unref();
    }
  }
  return s;
}

2887 2888 2889 2890 2891 2892 2893 2894 2895 2896 2897 2898 2899 2900 2901 2902 2903 2904 2905 2906 2907 2908
void DBImpl::NotifyOnExternalFileIngested(
    ColumnFamilyData* cfd, const ExternalSstFileIngestionJob& ingestion_job) {
#ifndef ROCKSDB_LITE
  if (immutable_db_options_.listeners.empty()) {
    return;
  }

  for (const IngestedFileInfo& f : ingestion_job.files_to_ingest()) {
    ExternalFileIngestionInfo info;
    info.cf_name = cfd->GetName();
    info.external_file_path = f.external_file_path;
    info.internal_file_path = f.internal_file_path;
    info.global_seqno = f.assigned_seqno;
    info.table_properties = f.table_properties;
    for (auto listener : immutable_db_options_.listeners) {
      listener->OnExternalFileIngested(this, info);
    }
  }

#endif
}

2909 2910 2911 2912 2913 2914 2915
void DBImpl::WaitForIngestFile() {
  mutex_.AssertHeld();
  while (num_running_ingest_file_ > 0) {
    bg_cv_.Wait();
  }
}

A
agiardullo 已提交
2916 2917
#endif  // ROCKSDB_LITE

2918
}  // namespace rocksdb