db_impl_compaction_flush.cc 120.1 KB
Newer Older
S
Siying Dong 已提交
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).
S
Siying Dong 已提交
5 6 7 8
//
// 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.
9
#include "db/db_impl/db_impl.h"
S
Siying Dong 已提交
10

11
#include <cinttypes>
S
Siying Dong 已提交
12 13

#include "db/builder.h"
14
#include "db/error_handler.h"
15
#include "db/event_helpers.h"
16
#include "file/sst_file_manager_impl.h"
17 18 19 20
#include "monitoring/iostats_context_imp.h"
#include "monitoring/perf_context_imp.h"
#include "monitoring/thread_status_updater.h"
#include "monitoring/thread_status_util.h"
21
#include "test_util/sync_point.h"
22
#include "util/cast_util.h"
23
#include "util/concurrent_task_limiter_impl.h"
S
Siying Dong 已提交
24

25
namespace ROCKSDB_NAMESPACE {
26

27
bool DBImpl::EnoughRoomForCompaction(
28
    ColumnFamilyData* cfd, const std::vector<CompactionInputFiles>& inputs,
29 30 31 32 33 34 35
    bool* sfm_reserved_compact_space, LogBuffer* log_buffer) {
  // Check if we have enough room to do the compaction
  bool enough_room = true;
#ifndef ROCKSDB_LITE
  auto sfm = static_cast<SstFileManagerImpl*>(
      immutable_db_options_.sst_file_manager.get());
  if (sfm) {
36 37 38 39 40
    // Pass the current bg_error_ to SFM so it can decide what checks to
    // perform. If this DB instance hasn't seen any error yet, the SFM can be
    // optimistic and not do disk space checks
    enough_room =
        sfm->EnoughRoomForCompaction(cfd, inputs, error_handler_.GetBGError());
41 42 43 44
    if (enough_room) {
      *sfm_reserved_compact_space = true;
    }
  }
45
#else
46
  (void)cfd;
47 48
  (void)inputs;
  (void)sfm_reserved_compact_space;
49 50 51 52 53 54 55 56 57 58 59 60
#endif  // ROCKSDB_LITE
  if (!enough_room) {
    // Just in case tests want to change the value of enough_room
    TEST_SYNC_POINT_CALLBACK(
        "DBImpl::BackgroundCompaction():CancelledCompaction", &enough_room);
    ROCKS_LOG_BUFFER(log_buffer,
                     "Cancelled compaction because not enough room");
    RecordTick(stats_, COMPACTION_CANCELLED, 1);
  }
  return enough_room;
}

Y
Yanqin Jin 已提交
61
bool DBImpl::RequestCompactionToken(ColumnFamilyData* cfd, bool force,
62 63 64 65 66 67 68 69 70 71 72
                                    std::unique_ptr<TaskLimiterToken>* token,
                                    LogBuffer* log_buffer) {
  assert(*token == nullptr);
  auto limiter = static_cast<ConcurrentTaskLimiterImpl*>(
      cfd->ioptions()->compaction_thread_limiter.get());
  if (limiter == nullptr) {
    return true;
  }
  *token = limiter->GetToken(force);
  if (*token != nullptr) {
    ROCKS_LOG_BUFFER(log_buffer,
Y
Yanqin Jin 已提交
73 74 75 76
                     "Thread limiter [%s] increase [%s] compaction task, "
                     "force: %s, tasks after: %d",
                     limiter->GetName().c_str(), cfd->GetName().c_str(),
                     force ? "true" : "false", limiter->GetOutstandingTask());
77 78 79 80 81
    return true;
  }
  return false;
}

82
IOStatus DBImpl::SyncClosedLogs(JobContext* job_context) {
S
Siying Dong 已提交
83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98
  TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Start");
  mutex_.AssertHeld();
  autovector<log::Writer*, 1> logs_to_sync;
  uint64_t current_log_number = logfile_number_;
  while (logs_.front().number < current_log_number &&
         logs_.front().getting_synced) {
    log_sync_cv_.Wait();
  }
  for (auto it = logs_.begin();
       it != logs_.end() && it->number < current_log_number; ++it) {
    auto& log = *it;
    assert(!log.getting_synced);
    log.getting_synced = true;
    logs_to_sync.push_back(log.writer);
  }

99
  IOStatus io_s;
S
Siying Dong 已提交
100 101 102 103 104 105 106
  if (!logs_to_sync.empty()) {
    mutex_.Unlock();

    for (log::Writer* log : logs_to_sync) {
      ROCKS_LOG_INFO(immutable_db_options_.info_log,
                     "[JOB %d] Syncing log #%" PRIu64, job_context->job_id,
                     log->get_log_number());
107 108
      io_s = log->file()->Sync(immutable_db_options_.use_fsync);
      if (!io_s.ok()) {
109 110
        break;
      }
111 112

      if (immutable_db_options_.recycle_log_file_num > 0) {
113 114
        io_s = log->Close();
        if (!io_s.ok()) {
115 116 117
          break;
        }
      }
S
Siying Dong 已提交
118
    }
119 120
    if (io_s.ok()) {
      io_s = directories_.GetWalDir()->Fsync(IOOptions(), nullptr);
S
Siying Dong 已提交
121 122 123 124 125 126
    }

    mutex_.Lock();

    // "number <= current_log_number - 1" is equivalent to
    // "number < current_log_number".
127 128 129
    MarkLogsSynced(current_log_number - 1, true, io_s);
    if (!io_s.ok()) {
      error_handler_.SetBGError(io_s, BackgroundErrorReason::kFlush);
S
Siying Dong 已提交
130
      TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Failed");
131
      return io_s;
S
Siying Dong 已提交
132 133
    }
  }
134
  return io_s;
S
Siying Dong 已提交
135 136 137 138
}

Status DBImpl::FlushMemTableToOutputFile(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
139
    bool* made_progress, JobContext* job_context,
140 141 142
    SuperVersionContext* superversion_context,
    std::vector<SequenceNumber>& snapshot_seqs,
    SequenceNumber earliest_write_conflict_snapshot,
143 144
    SnapshotChecker* snapshot_checker, LogBuffer* log_buffer,
    Env::Priority thread_pri) {
S
Siying Dong 已提交
145 146 147 148 149
  mutex_.AssertHeld();
  assert(cfd->imm()->NumNotFlushed() != 0);
  assert(cfd->imm()->IsFlushPending());

  FlushJob flush_job(
150
      dbname_, cfd, immutable_db_options_, mutable_cf_options,
151
      nullptr /* memtable_id */, file_options_for_compaction_, versions_.get(),
152 153 154
      &mutex_, &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot,
      snapshot_checker, job_context, log_buffer, directories_.GetDbDir(),
      GetDataDir(cfd, 0U),
S
Siying Dong 已提交
155
      GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_,
156
      &event_logger_, mutable_cf_options.report_bg_io_stats,
157 158
      true /* sync_output_directory */, true /* write_manifest */, thread_pri,
      db_id_, db_session_id_);
S
Siying Dong 已提交
159 160
  FileMetaData file_meta;

161
  TEST_SYNC_POINT("DBImpl::FlushMemTableToOutputFile:BeforePickMemtables");
S
Siying Dong 已提交
162
  flush_job.PickMemTable();
163
  TEST_SYNC_POINT("DBImpl::FlushMemTableToOutputFile:AfterPickMemtables");
S
Siying Dong 已提交
164

Y
Yi Wu 已提交
165 166
#ifndef ROCKSDB_LITE
  // may temporarily unlock and lock the mutex.
167
  NotifyOnFlushBegin(cfd, &file_meta, mutable_cf_options, job_context->job_id);
Y
Yi Wu 已提交
168 169
#endif  // ROCKSDB_LITE

S
Siying Dong 已提交
170
  Status s;
171
  IOStatus io_s;
S
Siying Dong 已提交
172
  if (logfile_number_ > 0 &&
173
      versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1) {
S
Siying Dong 已提交
174 175 176 177 178 179
    // If there are more than one column families, we need to make sure that
    // all the log files except the most recent one are synced. Otherwise if
    // the host crashes after flushing and before WAL is persistent, the
    // flushed SST may contain data from write batches whose updates to
    // other column families are missing.
    // SyncClosedLogs() may unlock and re-lock the db_mutex.
180 181
    io_s = SyncClosedLogs(job_context);
    s = io_s;
182 183
  } else {
    TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Skip");
S
Siying Dong 已提交
184 185 186 187 188 189 190 191 192
  }

  // Within flush_job.Run, rocksdb may call event listener to notify
  // file creation and deletion.
  //
  // Note that flush_job.Run will unlock and lock the db_mutex,
  // and EventListener callback will be called when the db_mutex
  // is unlocked by the current thread.
  if (s.ok()) {
S
Siying Dong 已提交
193
    s = flush_job.Run(&logs_with_prep_tracker_, &file_meta);
S
Siying Dong 已提交
194 195 196
  } else {
    flush_job.Cancel();
  }
197
  io_s = flush_job.io_status();
S
Siying Dong 已提交
198 199

  if (s.ok()) {
200 201
    InstallSuperVersionAndScheduleWork(cfd, superversion_context,
                                       mutable_cf_options);
S
Siying Dong 已提交
202
    if (made_progress) {
203
      *made_progress = true;
S
Siying Dong 已提交
204 205 206 207 208 209 210
    }
    VersionStorageInfo::LevelSummaryStorage tmp;
    ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n",
                     cfd->GetName().c_str(),
                     cfd->current()->storage_info()->LevelSummary(&tmp));
  }

211
  if (!s.ok() && !s.IsShutdownInProgress() && !s.IsColumnFamilyDropped()) {
212 213
    if (!io_s.ok() && !io_s.IsShutdownInProgress() &&
        !io_s.IsColumnFamilyDropped()) {
214 215 216 217 218
      error_handler_.SetBGError(io_s, BackgroundErrorReason::kFlush);
    } else {
      Status new_bg_error = s;
      error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
    }
S
Siying Dong 已提交
219 220 221 222
  }
  if (s.ok()) {
#ifndef ROCKSDB_LITE
    // may temporarily unlock and lock the mutex.
223 224
    NotifyOnFlushCompleted(cfd, mutable_cf_options,
                           flush_job.GetCommittedFlushJobsInfo());
S
Siying Dong 已提交
225 226 227 228 229
    auto sfm = static_cast<SstFileManagerImpl*>(
        immutable_db_options_.sst_file_manager.get());
    if (sfm) {
      // Notify sst_file_manager that a new file was added
      std::string file_path = MakeTableFileName(
230
          cfd->ioptions()->cf_paths[0].path, file_meta.fd.GetNumber());
S
Siying Dong 已提交
231
      sfm->OnAddFile(file_path);
232
      if (sfm->IsMaxAllowedSpaceReached()) {
233 234
        Status new_bg_error =
            Status::SpaceLimit("Max allowed space was reached");
S
Siying Dong 已提交
235 236
        TEST_SYNC_POINT_CALLBACK(
            "DBImpl::FlushMemTableToOutputFile:MaxAllowedSpaceReached",
237
            &new_bg_error);
238
        error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
S
Siying Dong 已提交
239 240 241 242
      }
    }
#endif  // ROCKSDB_LITE
  }
243
  TEST_SYNC_POINT("DBImpl::FlushMemTableToOutputFile:Finish");
S
Siying Dong 已提交
244 245 246
  return s;
}

247 248
Status DBImpl::FlushMemTablesToOutputFiles(
    const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
249
    JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri) {
250
  if (immutable_db_options_.atomic_flush) {
251 252
    return AtomicFlushMemTablesToOutputFiles(
        bg_flush_args, made_progress, job_context, log_buffer, thread_pri);
Y
Yanqin Jin 已提交
253
  }
254 255 256 257 258
  std::vector<SequenceNumber> snapshot_seqs;
  SequenceNumber earliest_write_conflict_snapshot;
  SnapshotChecker* snapshot_checker;
  GetSnapshotContext(job_context, &snapshot_seqs,
                     &earliest_write_conflict_snapshot, &snapshot_checker);
259
  Status status;
260 261
  for (auto& arg : bg_flush_args) {
    ColumnFamilyData* cfd = arg.cfd_;
262
    MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
263
    SuperVersionContext* superversion_context = arg.superversion_context_;
264 265 266
    Status s = FlushMemTableToOutputFile(
        cfd, mutable_cf_options, made_progress, job_context,
        superversion_context, snapshot_seqs, earliest_write_conflict_snapshot,
267
        snapshot_checker, log_buffer, thread_pri);
268
    if (!s.ok()) {
269
      status = s;
270
      if (!s.IsShutdownInProgress() && !s.IsColumnFamilyDropped()) {
271 272 273 274
        // At this point, DB is not shutting down, nor is cfd dropped.
        // Something is wrong, thus we break out of the loop.
        break;
      }
275 276
    }
  }
277
  return status;
278 279
}

280 281 282 283 284 285 286 287 288 289 290
/*
 * Atomically flushes multiple column families.
 *
 * For each column family, all memtables with ID smaller than or equal to the
 * ID specified in bg_flush_args will be flushed. Only after all column
 * families finish flush will this function commit to MANIFEST. If any of the
 * column families are not flushed successfully, this function does not have
 * any side-effect on the state of the database.
 */
Status DBImpl::AtomicFlushMemTablesToOutputFiles(
    const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
291
    JobContext* job_context, LogBuffer* log_buffer, Env::Priority thread_pri) {
292 293 294 295 296 297 298 299 300 301 302 303 304 305
  mutex_.AssertHeld();

  autovector<ColumnFamilyData*> cfds;
  for (const auto& arg : bg_flush_args) {
    cfds.emplace_back(arg.cfd_);
  }

#ifndef NDEBUG
  for (const auto cfd : cfds) {
    assert(cfd->imm()->NumNotFlushed() != 0);
    assert(cfd->imm()->IsFlushPending());
  }
#endif /* !NDEBUG */

306
  std::vector<SequenceNumber> snapshot_seqs;
307
  SequenceNumber earliest_write_conflict_snapshot;
308 309 310
  SnapshotChecker* snapshot_checker;
  GetSnapshotContext(job_context, &snapshot_seqs,
                     &earliest_write_conflict_snapshot, &snapshot_checker);
311

312
  autovector<FSDirectory*> distinct_output_dirs;
313
  autovector<std::string> distinct_output_dir_paths;
314
  std::vector<std::unique_ptr<FlushJob>> jobs;
315
  std::vector<MutableCFOptions> all_mutable_cf_options;
316
  int num_cfs = static_cast<int>(cfds.size());
317
  all_mutable_cf_options.reserve(num_cfs);
318 319
  for (int i = 0; i < num_cfs; ++i) {
    auto cfd = cfds[i];
320
    FSDirectory* data_dir = GetDataDir(cfd, 0U);
321
    const std::string& curr_path = cfd->ioptions()->cf_paths[0].path;
322 323 324 325 326

    // Add to distinct output directories if eligible. Use linear search. Since
    // the number of elements in the vector is not large, performance should be
    // tolerable.
    bool found = false;
327 328
    for (const auto& path : distinct_output_dir_paths) {
      if (path == curr_path) {
329 330 331 332 333
        found = true;
        break;
      }
    }
    if (!found) {
334
      distinct_output_dir_paths.emplace_back(curr_path);
335 336 337
      distinct_output_dirs.emplace_back(data_dir);
    }

338 339
    all_mutable_cf_options.emplace_back(*cfd->GetLatestMutableCFOptions());
    const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.back();
340
    const uint64_t* max_memtable_id = &(bg_flush_args[i].max_memtable_id_);
341
    jobs.emplace_back(new FlushJob(
342
        dbname_, cfd, immutable_db_options_, mutable_cf_options,
343
        max_memtable_id, file_options_for_compaction_, versions_.get(), &mutex_,
344 345 346 347
        &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot,
        snapshot_checker, job_context, log_buffer, directories_.GetDbDir(),
        data_dir, GetCompressionFlush(*cfd->ioptions(), mutable_cf_options),
        stats_, &event_logger_, mutable_cf_options.report_bg_io_stats,
348
        false /* sync_output_directory */, false /* write_manifest */,
349
        thread_pri, db_id_, db_session_id_));
350
    jobs.back()->PickMemTable();
351 352
  }

353
  std::vector<FileMetaData> file_meta(num_cfs);
354
  Status s;
355
  IOStatus io_s;
356 357 358
  assert(num_cfs == static_cast<int>(jobs.size()));

#ifndef ROCKSDB_LITE
359 360
  for (int i = 0; i != num_cfs; ++i) {
    const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.at(i);
361 362
    // may temporarily unlock and lock the mutex.
    NotifyOnFlushBegin(cfds[i], &file_meta[i], mutable_cf_options,
363
                       job_context->job_id);
364
  }
365
#endif /* !ROCKSDB_LITE */
366 367 368 369

  if (logfile_number_ > 0) {
    // TODO (yanqin) investigate whether we should sync the closed logs for
    // single column family case.
370 371
    io_s = SyncClosedLogs(job_context);
    s = io_s;
372 373
  }

Y
Yanqin Jin 已提交
374 375 376
  // exec_status stores the execution status of flush_jobs as
  // <bool /* executed */, Status /* status code */>
  autovector<std::pair<bool, Status>> exec_status;
377
  autovector<IOStatus> io_status;
Y
Yanqin Jin 已提交
378 379
  for (int i = 0; i != num_cfs; ++i) {
    // Initially all jobs are not executed, with status OK.
380
    exec_status.emplace_back(false, Status::OK());
381
    io_status.emplace_back(IOStatus::OK());
Y
Yanqin Jin 已提交
382 383
  }

384 385
  if (s.ok()) {
    // TODO (yanqin): parallelize jobs with threads.
386
    for (int i = 1; i != num_cfs; ++i) {
Y
Yanqin Jin 已提交
387
      exec_status[i].second =
388
          jobs[i]->Run(&logs_with_prep_tracker_, &file_meta[i]);
Y
Yanqin Jin 已提交
389
      exec_status[i].first = true;
390
      io_status[i] = jobs[i]->io_status();
391
    }
392 393 394 395 396 397
    if (num_cfs > 1) {
      TEST_SYNC_POINT(
          "DBImpl::AtomicFlushMemTablesToOutputFiles:SomeFlushJobsComplete:1");
      TEST_SYNC_POINT(
          "DBImpl::AtomicFlushMemTablesToOutputFiles:SomeFlushJobsComplete:2");
    }
398 399
    assert(exec_status.size() > 0);
    assert(!file_meta.empty());
400
    exec_status[0].second =
401
        jobs[0]->Run(&logs_with_prep_tracker_, &file_meta[0]);
402
    exec_status[0].first = true;
403
    io_status[0] = jobs[0]->io_status();
404 405 406 407 408

    Status error_status;
    for (const auto& e : exec_status) {
      if (!e.second.ok()) {
        s = e.second;
409 410
        if (!e.second.IsShutdownInProgress() &&
            !e.second.IsColumnFamilyDropped()) {
411 412 413 414 415
          // If a flush job did not return OK, and the CF is not dropped, and
          // the DB is not shutting down, then we have to return this result to
          // caller later.
          error_status = e.second;
        }
416 417
      }
    }
418 419

    s = error_status.ok() ? s : error_status;
420 421
  }

422 423 424 425 426 427 428 429 430 431 432 433 434 435
  if (io_s.ok()) {
    IOStatus io_error = IOStatus::OK();
    for (int i = 0; i != static_cast<int>(io_status.size()); i++) {
      if (!io_status[i].ok() && !io_status[i].IsShutdownInProgress() &&
          !io_status[i].IsColumnFamilyDropped()) {
        io_error = io_status[i];
      }
    }
    io_s = io_error;
    if (s.ok() && !io_s.ok()) {
      s = io_s;
    }
  }

436
  if (s.IsColumnFamilyDropped()) {
437 438 439
    s = Status::OK();
  }

440
  if (s.ok() || s.IsShutdownInProgress()) {
441 442 443
    // Sync on all distinct output directories.
    for (auto dir : distinct_output_dirs) {
      if (dir != nullptr) {
444
        Status error_status = dir->Fsync(IOOptions(), nullptr);
445 446
        if (!error_status.ok()) {
          s = error_status;
447 448 449 450
          break;
        }
      }
    }
451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467
  } else {
    // Need to undo atomic flush if something went wrong, i.e. s is not OK and
    // it is not because of CF drop.
    // Have to cancel the flush jobs that have NOT executed because we need to
    // unref the versions.
    for (int i = 0; i != num_cfs; ++i) {
      if (!exec_status[i].first) {
        jobs[i]->Cancel();
      }
    }
    for (int i = 0; i != num_cfs; ++i) {
      if (exec_status[i].first && exec_status[i].second.ok()) {
        auto& mems = jobs[i]->GetMemTables();
        cfds[i]->imm()->RollbackMemtableFlush(mems,
                                              file_meta[i].fd.GetNumber());
      }
    }
468
  }
469

470 471 472 473
  if (s.ok()) {
    auto wait_to_install_func = [&]() {
      bool ready = true;
      for (size_t i = 0; i != cfds.size(); ++i) {
474
        const auto& mems = jobs[i]->GetMemTables();
475 476
        if (cfds[i]->IsDropped()) {
          // If the column family is dropped, then do not wait.
477
          continue;
478 479 480 481 482 483 484 485 486 487 488 489 490 491 492
        } else if (!mems.empty() &&
                   cfds[i]->imm()->GetEarliestMemTableID() < mems[0]->GetID()) {
          // If a flush job needs to install the flush result for mems and
          // mems[0] is not the earliest memtable, it means another thread must
          // be installing flush results for the same column family, then the
          // current thread needs to wait.
          ready = false;
          break;
        } else if (mems.empty() && cfds[i]->imm()->GetEarliestMemTableID() <=
                                       bg_flush_args[i].max_memtable_id_) {
          // If a flush job does not need to install flush results, then it has
          // to wait until all memtables up to max_memtable_id_ (inclusive) are
          // installed.
          ready = false;
          break;
493
        }
494
      }
495 496 497 498 499 500 501 502 503 504 505 506 507
      return ready;
    };

    bool resuming_from_bg_err = error_handler_.IsDBStopped();
    while ((!error_handler_.IsDBStopped() ||
            error_handler_.GetRecoveryError().ok()) &&
           !wait_to_install_func()) {
      atomic_flush_install_cv_.Wait();
    }

    s = resuming_from_bg_err ? error_handler_.GetRecoveryError()
                             : error_handler_.GetBGError();
  }
508

509 510 511 512
  if (s.ok()) {
    autovector<ColumnFamilyData*> tmp_cfds;
    autovector<const autovector<MemTable*>*> mems_list;
    autovector<const MutableCFOptions*> mutable_cf_options_list;
513
    autovector<FileMetaData*> tmp_file_meta;
514
    for (int i = 0; i != num_cfs; ++i) {
515
      const auto& mems = jobs[i]->GetMemTables();
516 517 518
      if (!cfds[i]->IsDropped() && !mems.empty()) {
        tmp_cfds.emplace_back(cfds[i]);
        mems_list.emplace_back(&mems);
519
        mutable_cf_options_list.emplace_back(&all_mutable_cf_options[i]);
520
        tmp_file_meta.emplace_back(&file_meta[i]);
521
      }
522
    }
523 524 525

    s = InstallMemtableAtomicFlushResults(
        nullptr /* imm_lists */, tmp_cfds, mutable_cf_options_list, mems_list,
526 527
        versions_.get(), &mutex_, tmp_file_meta,
        &job_context->memtables_to_free, directories_.GetDbDir(), log_buffer);
528 529
  }

530
  if (s.ok()) {
531 532 533
    assert(num_cfs ==
           static_cast<int>(job_context->superversion_contexts.size()));
    for (int i = 0; i != num_cfs; ++i) {
534 535 536
      if (cfds[i]->IsDropped()) {
        continue;
      }
537 538
      InstallSuperVersionAndScheduleWork(cfds[i],
                                         &job_context->superversion_contexts[i],
539
                                         all_mutable_cf_options[i]);
540 541 542 543 544 545 546 547 548 549 550
      VersionStorageInfo::LevelSummaryStorage tmp;
      ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n",
                       cfds[i]->GetName().c_str(),
                       cfds[i]->current()->storage_info()->LevelSummary(&tmp));
    }
    if (made_progress) {
      *made_progress = true;
    }
#ifndef ROCKSDB_LITE
    auto sfm = static_cast<SstFileManagerImpl*>(
        immutable_db_options_.sst_file_manager.get());
551
    assert(all_mutable_cf_options.size() == static_cast<size_t>(num_cfs));
552
    for (int i = 0; i != num_cfs; ++i) {
553 554 555
      if (cfds[i]->IsDropped()) {
        continue;
      }
556 557
      NotifyOnFlushCompleted(cfds[i], all_mutable_cf_options[i],
                             jobs[i]->GetCommittedFlushJobsInfo());
558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573
      if (sfm) {
        std::string file_path = MakeTableFileName(
            cfds[i]->ioptions()->cf_paths[0].path, file_meta[i].fd.GetNumber());
        sfm->OnAddFile(file_path);
        if (sfm->IsMaxAllowedSpaceReached() &&
            error_handler_.GetBGError().ok()) {
          Status new_bg_error =
              Status::SpaceLimit("Max allowed space was reached");
          error_handler_.SetBGError(new_bg_error,
                                    BackgroundErrorReason::kFlush);
        }
      }
    }
#endif  // ROCKSDB_LITE
  }

574 575 576
  // Need to undo atomic flush if something went wrong, i.e. s is not OK and
  // it is not because of CF drop.
  if (!s.ok() && !s.IsColumnFamilyDropped()) {
577
    if (!io_s.ok() && !io_s.IsColumnFamilyDropped()) {
578 579 580 581 582
      error_handler_.SetBGError(io_s, BackgroundErrorReason::kFlush);
    } else {
      Status new_bg_error = s;
      error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
    }
583 584 585 586 587
  }

  return s;
}

Y
Yi Wu 已提交
588 589
void DBImpl::NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta,
                                const MutableCFOptions& mutable_cf_options,
590
                                int job_id) {
Y
Yi Wu 已提交
591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606 607
#ifndef ROCKSDB_LITE
  if (immutable_db_options_.listeners.size() == 0U) {
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  bool triggered_writes_slowdown =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
       mutable_cf_options.level0_slowdown_writes_trigger);
  bool triggered_writes_stop =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
       mutable_cf_options.level0_stop_writes_trigger);
  // release lock while notifying events
  mutex_.Unlock();
  {
608
    FlushJobInfo info{};
609
    info.cf_id = cfd->GetID();
Y
Yi Wu 已提交
610 611 612
    info.cf_name = cfd->GetName();
    // TODO(yhchiang): make db_paths dynamic in case flush does not
    //                 go to L0 in the future.
613 614 615 616
    const uint64_t file_number = file_meta->fd.GetNumber();
    info.file_path =
        MakeTableFileName(cfd->ioptions()->cf_paths[0].path, file_number);
    info.file_number = file_number;
Y
Yi Wu 已提交
617 618 619 620
    info.thread_id = env_->GetThreadID();
    info.job_id = job_id;
    info.triggered_writes_slowdown = triggered_writes_slowdown;
    info.triggered_writes_stop = triggered_writes_stop;
621 622
    info.smallest_seqno = file_meta->fd.smallest_seqno;
    info.largest_seqno = file_meta->fd.largest_seqno;
623
    info.flush_reason = cfd->GetFlushReason();
Y
Yi Wu 已提交
624 625 626 627 628 629 630
    for (auto listener : immutable_db_options_.listeners) {
      listener->OnFlushBegin(this, info);
    }
  }
  mutex_.Lock();
// no need to signal bg_cv_ as it will be signaled at the end of the
// flush process.
631 632 633 634 635
#else
  (void)cfd;
  (void)file_meta;
  (void)mutable_cf_options;
  (void)job_id;
Y
Yi Wu 已提交
636 637 638
#endif  // ROCKSDB_LITE
}

639 640 641
void DBImpl::NotifyOnFlushCompleted(
    ColumnFamilyData* cfd, const MutableCFOptions& mutable_cf_options,
    std::list<std::unique_ptr<FlushJobInfo>>* flush_jobs_info) {
S
Siying Dong 已提交
642
#ifndef ROCKSDB_LITE
643
  assert(flush_jobs_info != nullptr);
S
Siying Dong 已提交
644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659
  if (immutable_db_options_.listeners.size() == 0U) {
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
  bool triggered_writes_slowdown =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
       mutable_cf_options.level0_slowdown_writes_trigger);
  bool triggered_writes_stop =
      (cfd->current()->storage_info()->NumLevelFiles(0) >=
       mutable_cf_options.level0_stop_writes_trigger);
  // release lock while notifying events
  mutex_.Unlock();
  {
660 661 662 663 664 665
    for (auto& info : *flush_jobs_info) {
      info->triggered_writes_slowdown = triggered_writes_slowdown;
      info->triggered_writes_stop = triggered_writes_stop;
      for (auto listener : immutable_db_options_.listeners) {
        listener->OnFlushCompleted(this, *info);
      }
S
Siying Dong 已提交
666
    }
667
    flush_jobs_info->clear();
S
Siying Dong 已提交
668 669 670 671
  }
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
672 673 674
#else
  (void)cfd;
  (void)mutable_cf_options;
675
  (void)flush_jobs_info;
S
Siying Dong 已提交
676 677 678 679 680 681
#endif  // ROCKSDB_LITE
}

Status DBImpl::CompactRange(const CompactRangeOptions& options,
                            ColumnFamilyHandle* column_family,
                            const Slice* begin, const Slice* end) {
682 683 684 685
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

  if (options.target_path_id >= cfd->ioptions()->cf_paths.size()) {
S
Siying Dong 已提交
686 687 688
    return Status::InvalidArgument("Invalid target path ID");
  }

689
  bool flush_needed = true;
690 691 692 693 694
  if (begin != nullptr && end != nullptr) {
    // TODO(ajkr): We could also optimize away the flush in certain cases where
    // one/both sides of the interval are unbounded. But it requires more
    // changes to RangesOverlapWithMemtables.
    Range range(*begin, *end);
695
    SuperVersion* super_version = cfd->GetReferencedSuperVersion(this);
696 697 698 699
    cfd->RangesOverlapWithMemtables({range}, super_version, &flush_needed);
    CleanupSuperVersion(super_version);
  }

700 701
  Status s;
  if (flush_needed) {
702 703
    FlushOptions fo;
    fo.allow_write_stall = options.allow_write_stall;
704
    if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
705
      autovector<ColumnFamilyData*> cfds;
706
      mutex_.Lock();
Y
Yanqin Jin 已提交
707
      SelectColumnFamiliesForAtomicFlush(&cfds);
708
      mutex_.Unlock();
Y
Yanqin Jin 已提交
709 710 711 712 713 714
      s = AtomicFlushMemTables(cfds, fo, FlushReason::kManualCompaction,
                               false /* writes_stopped */);
    } else {
      s = FlushMemTable(cfd, fo, FlushReason::kManualCompaction,
                        false /* writes_stopped*/);
    }
715 716 717 718
    if (!s.ok()) {
      LogFlush(immutable_db_options_.info_log);
      return s;
    }
S
Siying Dong 已提交
719 720
  }

721 722 723
  constexpr int kInvalidLevel = -1;
  int final_output_level = kInvalidLevel;
  bool exclusive = options.exclusive_manual_compaction;
S
Siying Dong 已提交
724 725 726
  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
727 728 729 730 731
    final_output_level = cfd->NumberLevels() - 1;
    // if bottom most level is reserved
    if (immutable_db_options_.allow_ingest_behind) {
      final_output_level--;
    }
S
Siying Dong 已提交
732
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
733
                            final_output_level, options, begin, end, exclusive,
734
                            false, port::kMaxUint64);
S
Siying Dong 已提交
735
  } else {
736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757
    int first_overlapped_level = kInvalidLevel;
    int max_overlapped_level = kInvalidLevel;
    {
      SuperVersion* super_version = cfd->GetReferencedSuperVersion(this);
      Version* current_version = super_version->current;
      ReadOptions ro;
      ro.total_order_seek = true;
      bool overlap;
      for (int level = 0;
           level < current_version->storage_info()->num_non_empty_levels();
           level++) {
        overlap = true;
        if (begin != nullptr && end != nullptr) {
          Status status = current_version->OverlapWithLevelIterator(
              ro, file_options_, *begin, *end, level, &overlap);
          if (!status.ok()) {
            overlap = current_version->storage_info()->OverlapInLevel(
                level, begin, end);
          }
        } else {
          overlap = current_version->storage_info()->OverlapInLevel(level,
                                                                    begin, end);
S
Siying Dong 已提交
758
        }
759 760 761 762 763
        if (overlap) {
          if (first_overlapped_level == kInvalidLevel) {
            first_overlapped_level = level;
          }
          max_overlapped_level = level;
S
Siying Dong 已提交
764 765
        }
      }
766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821
      CleanupSuperVersion(super_version);
    }
    if (s.ok() && first_overlapped_level != kInvalidLevel) {
      // max_file_num_to_ignore can be used to filter out newly created SST
      // files, useful for bottom level compaction in a manual compaction
      uint64_t max_file_num_to_ignore = port::kMaxUint64;
      uint64_t next_file_number = versions_->current_next_file_number();
      final_output_level = max_overlapped_level;
      int output_level;
      for (int level = first_overlapped_level; level <= max_overlapped_level;
           level++) {
        // in case the compaction is universal or if we're compacting the
        // bottom-most level, the output level will be the same as input one.
        // level 0 can never be the bottommost level (i.e. if all files are in
        // level 0, we will compact to level 1)
        if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
            cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
          output_level = level;
        } else if (level == max_overlapped_level && level > 0) {
          if (options.bottommost_level_compaction ==
              BottommostLevelCompaction::kSkip) {
            // Skip bottommost level compaction
            continue;
          } else if (options.bottommost_level_compaction ==
                         BottommostLevelCompaction::kIfHaveCompactionFilter &&
                     cfd->ioptions()->compaction_filter == nullptr &&
                     cfd->ioptions()->compaction_filter_factory == nullptr) {
            // Skip bottommost level compaction since we don't have a compaction
            // filter
            continue;
          }
          output_level = level;
          // update max_file_num_to_ignore only for bottom level compaction
          // because data in newly compacted files in middle levels may still
          // need to be pushed down
          max_file_num_to_ignore = next_file_number;
        } else {
          output_level = level + 1;
          if (cfd->ioptions()->compaction_style == kCompactionStyleLevel &&
              cfd->ioptions()->level_compaction_dynamic_level_bytes &&
              level == 0) {
            output_level = ColumnFamilyData::kCompactToBaseLevel;
          }
        }
        s = RunManualCompaction(cfd, level, output_level, options, begin, end,
                                exclusive, false, max_file_num_to_ignore);
        if (!s.ok()) {
          break;
        }
        if (output_level == ColumnFamilyData::kCompactToBaseLevel) {
          final_output_level = cfd->NumberLevels() - 1;
        } else if (output_level > final_output_level) {
          final_output_level = output_level;
        }
        TEST_SYNC_POINT("DBImpl::RunManualCompaction()::1");
        TEST_SYNC_POINT("DBImpl::RunManualCompaction()::2");
S
Siying Dong 已提交
822 823 824
      }
    }
  }
825
  if (!s.ok() || final_output_level == kInvalidLevel) {
S
Siying Dong 已提交
826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850
    LogFlush(immutable_db_options_.info_log);
    return s;
  }

  if (options.change_level) {
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "[RefitLevel] waiting for background threads to stop");
    s = PauseBackgroundWork();
    if (s.ok()) {
      s = ReFitLevel(cfd, final_output_level, options.target_level);
    }
    ContinueBackgroundWork();
  }
  LogFlush(immutable_db_options_.info_log);

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

  return s;
}

851 852 853 854
Status DBImpl::CompactFiles(const CompactionOptions& compact_options,
                            ColumnFamilyHandle* column_family,
                            const std::vector<std::string>& input_file_names,
                            const int output_level, const int output_path_id,
855 856
                            std::vector<std::string>* const output_file_names,
                            CompactionJobInfo* compaction_job_info) {
S
Siying Dong 已提交
857
#ifdef ROCKSDB_LITE
858 859 860 861 862 863
  (void)compact_options;
  (void)column_family;
  (void)input_file_names;
  (void)output_level;
  (void)output_path_id;
  (void)output_file_names;
864
  (void)compaction_job_info;
865
  // not supported in lite version
S
Siying Dong 已提交
866 867 868 869 870 871 872 873 874 875 876 877 878 879 880
  return Status::NotSupported("Not supported in ROCKSDB LITE");
#else
  if (column_family == nullptr) {
    return Status::InvalidArgument("ColumnFamilyHandle must be non-null.");
  }

  auto cfd = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family)->cfd();
  assert(cfd);

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

  // Perform CompactFiles
D
DorianZheng 已提交
881
  TEST_SYNC_POINT("TestCompactFiles::IngestExternalFile2");
S
Siying Dong 已提交
882 883 884 885 886 887 888
  {
    InstrumentedMutexLock l(&mutex_);

    // This call will unlock/lock the mutex to wait for current running
    // IngestExternalFile() calls to finish.
    WaitForIngestFile();

D
DorianZheng 已提交
889 890 891 892 893 894
    // We need to get current after `WaitForIngestFile`, because
    // `IngestExternalFile` may add files that overlap with `input_file_names`
    auto* current = cfd->current();
    current->Ref();

    s = CompactFilesImpl(compact_options, cfd, current, input_file_names,
895
                         output_file_names, output_level, output_path_id,
896
                         &job_context, &log_buffer, compaction_job_info);
D
DorianZheng 已提交
897 898

    current->Unref();
S
Siying Dong 已提交
899 900 901 902 903 904 905 906 907 908 909 910 911
  }

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

  // delete unnecessary files if any, this is done outside the mutex
912 913
  if (job_context.HaveSomethingToClean() ||
      job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
S
Siying Dong 已提交
914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934
    // Have to flush the info logs before bg_compaction_scheduled_--
    // because if bg_flush_scheduled_ becomes 0 and the lock is
    // released, the deconstructor of DB can kick in and destroy all the
    // states of DB so info_log might not be available after that point.
    // It also applies to access other states that DB owns.
    log_buffer.FlushBufferToLog();
    if (job_context.HaveSomethingToDelete()) {
      // no mutex is locked here.  No need to Unlock() and Lock() here.
      PurgeObsoleteFiles(job_context);
    }
    job_context.Clean();
  }

  return s;
#endif  // ROCKSDB_LITE
}

#ifndef ROCKSDB_LITE
Status DBImpl::CompactFilesImpl(
    const CompactionOptions& compact_options, ColumnFamilyData* cfd,
    Version* version, const std::vector<std::string>& input_file_names,
935
    std::vector<std::string>* const output_file_names, const int output_level,
936 937
    int output_path_id, JobContext* job_context, LogBuffer* log_buffer,
    CompactionJobInfo* compaction_job_info) {
S
Siying Dong 已提交
938 939 940 941 942
  mutex_.AssertHeld();

  if (shutting_down_.load(std::memory_order_acquire)) {
    return Status::ShutdownInProgress();
  }
943 944 945
  if (manual_compaction_paused_.load(std::memory_order_acquire)) {
    return Status::Incomplete(Status::SubCode::kManualCompactionPaused);
  }
S
Siying Dong 已提交
946 947

  std::unordered_set<uint64_t> input_set;
948
  for (const auto& file_name : input_file_names) {
S
Siying Dong 已提交
949 950 951 952 953 954 955 956 957
    input_set.insert(TableFileNameToNumber(file_name));
  }

  ColumnFamilyMetaData cf_meta;
  // TODO(yhchiang): can directly use version here if none of the
  // following functions call is pluggable to external developers.
  version->GetColumnFamilyMetaData(&cf_meta);

  if (output_path_id < 0) {
958
    if (cfd->ioptions()->cf_paths.size() == 1U) {
S
Siying Dong 已提交
959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977 978 979
      output_path_id = 0;
    } else {
      return Status::NotSupported(
          "Automatic output path selection is not "
          "yet supported in CompactFiles()");
    }
  }

  Status s = cfd->compaction_picker()->SanitizeCompactionInputFiles(
      &input_set, cf_meta, output_level);
  if (!s.ok()) {
    return s;
  }

  std::vector<CompactionInputFiles> input_files;
  s = cfd->compaction_picker()->GetCompactionInputsFromFileNumbers(
      &input_files, &input_set, version->storage_info(), compact_options);
  if (!s.ok()) {
    return s;
  }

980
  for (const auto& inputs : input_files) {
S
Siying Dong 已提交
981
    if (cfd->compaction_picker()->AreFilesInCompaction(inputs.files)) {
S
Siying Dong 已提交
982 983 984 985 986
      return Status::Aborted(
          "Some of the necessary compaction input "
          "files are already being compacted");
    }
  }
987 988 989
  bool sfm_reserved_compact_space = false;
  // First check if we have enough room to do the compaction
  bool enough_room = EnoughRoomForCompaction(
990
      cfd, input_files, &sfm_reserved_compact_space, log_buffer);
991 992 993 994 995 996

  if (!enough_room) {
    // m's vars will get set properly at the end of this function,
    // as long as status == CompactionTooLarge
    return Status::CompactionTooLarge();
  }
S
Siying Dong 已提交
997 998 999 1000

  // At this point, CompactFiles will be run.
  bg_compaction_scheduled_++;

1001
  std::unique_ptr<Compaction> c;
S
Siying Dong 已提交
1002
  assert(cfd->compaction_picker());
S
Siying Dong 已提交
1003
  c.reset(cfd->compaction_picker()->CompactFiles(
S
Siying Dong 已提交
1004 1005
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
1006 1007 1008 1009
  // we already sanitized the set of input files and checked for conflicts
  // without releasing the lock, so we're guaranteed a compaction can be formed.
  assert(c != nullptr);

S
Siying Dong 已提交
1010 1011 1012 1013
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
  assert(!c->deletion_compaction());

1014
  std::vector<SequenceNumber> snapshot_seqs;
S
Siying Dong 已提交
1015
  SequenceNumber earliest_write_conflict_snapshot;
1016 1017 1018
  SnapshotChecker* snapshot_checker;
  GetSnapshotContext(job_context, &snapshot_seqs,
                     &earliest_write_conflict_snapshot, &snapshot_checker);
S
Siying Dong 已提交
1019

1020 1021 1022
  std::unique_ptr<std::list<uint64_t>::iterator> pending_outputs_inserted_elem(
      new std::list<uint64_t>::iterator(
          CaptureCurrentFileNumberInPendingOutputs()));
S
Siying Dong 已提交
1023 1024

  assert(is_snapshot_supported_ || snapshots_.empty());
1025
  CompactionJobStats compaction_job_stats;
S
Siying Dong 已提交
1026
  CompactionJob compaction_job(
1027
      job_context->job_id, c.get(), immutable_db_options_,
1028
      file_options_for_compaction_, versions_.get(), &shutting_down_,
1029
      preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
1030
      GetDataDir(c->column_family_data(), c->output_path_id()), stats_, &mutex_,
1031
      &error_handler_, snapshot_seqs, earliest_write_conflict_snapshot,
1032
      snapshot_checker, table_cache_, &event_logger_,
1033
      c->mutable_cf_options()->paranoid_file_checks,
S
Siying Dong 已提交
1034
      c->mutable_cf_options()->report_bg_io_stats, dbname_,
1035 1036
      &compaction_job_stats, Env::Priority::USER, &manual_compaction_paused_,
      db_id_, db_session_id_);
S
Siying Dong 已提交
1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056

  // Creating a compaction influences the compaction score because the score
  // takes running compactions into account (by skipping files that are already
  // being compacted). Since we just changed compaction score, we recalculate it
  // here.
  version->storage_info()->ComputeCompactionScore(*cfd->ioptions(),
                                                  *c->mutable_cf_options());

  compaction_job.Prepare();

  mutex_.Unlock();
  TEST_SYNC_POINT("CompactFilesImpl:0");
  TEST_SYNC_POINT("CompactFilesImpl:1");
  compaction_job.Run();
  TEST_SYNC_POINT("CompactFilesImpl:2");
  TEST_SYNC_POINT("CompactFilesImpl:3");
  mutex_.Lock();

  Status status = compaction_job.Install(*c->mutable_cf_options());
  if (status.ok()) {
Y
Yanqin Jin 已提交
1057 1058 1059
    InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                       &job_context->superversion_contexts[0],
                                       *c->mutable_cf_options());
S
Siying Dong 已提交
1060 1061
  }
  c->ReleaseCompactionFiles(s);
1062 1063 1064 1065 1066 1067 1068 1069
#ifndef ROCKSDB_LITE
  // Need to make sure SstFileManager does its bookkeeping
  auto sfm = static_cast<SstFileManagerImpl*>(
      immutable_db_options_.sst_file_manager.get());
  if (sfm && sfm_reserved_compact_space) {
    sfm->OnCompactionCompletion(c.get());
  }
#endif  // ROCKSDB_LITE
S
Siying Dong 已提交
1070 1071 1072

  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

1073 1074 1075 1076 1077
  if (compaction_job_info != nullptr) {
    BuildCompactionJobInfo(cfd, c.get(), s, compaction_job_stats,
                           job_context->job_id, version, compaction_job_info);
  }

S
Siying Dong 已提交
1078 1079
  if (status.ok()) {
    // Done
1080
  } else if (status.IsColumnFamilyDropped() || status.IsShutdownInProgress()) {
S
Siying Dong 已提交
1081
    // Ignore compaction errors found during shutting down
1082 1083 1084 1085 1086 1087
  } else if (status.IsManualCompactionPaused()) {
    // Don't report stopping manual compaction as error
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "[%s] [JOB %d] Stopping manual compaction",
                   c->column_family_data()->GetName().c_str(),
                   job_context->job_id);
S
Siying Dong 已提交
1088 1089 1090 1091 1092
  } else {
    ROCKS_LOG_WARN(immutable_db_options_.info_log,
                   "[%s] [JOB %d] Compaction error: %s",
                   c->column_family_data()->GetName().c_str(),
                   job_context->job_id, status.ToString().c_str());
1093
    error_handler_.SetBGError(status, BackgroundErrorReason::kCompaction);
S
Siying Dong 已提交
1094 1095
  }

1096
  if (output_file_names != nullptr) {
1097
    for (const auto& newf : c->edit()->GetNewFiles()) {
1098 1099 1100 1101
      (*output_file_names)
          .push_back(TableFileName(c->immutable_cf_options()->cf_paths,
                                   newf.second.fd.GetNumber(),
                                   newf.second.fd.GetPathId()));
1102 1103 1104
    }
  }

S
Siying Dong 已提交
1105 1106 1107 1108 1109 1110
  c.reset();

  bg_compaction_scheduled_--;
  if (bg_compaction_scheduled_ == 0) {
    bg_cv_.SignalAll();
  }
1111
  MaybeScheduleFlushOrCompaction();
1112
  TEST_SYNC_POINT("CompactFilesImpl:End");
S
Siying Dong 已提交
1113 1114 1115 1116 1117 1118 1119 1120

  return status;
}
#endif  // ROCKSDB_LITE

Status DBImpl::PauseBackgroundWork() {
  InstrumentedMutexLock guard_lock(&mutex_);
  bg_compaction_paused_++;
1121 1122
  while (bg_bottom_compaction_scheduled_ > 0 || bg_compaction_scheduled_ > 0 ||
         bg_flush_scheduled_ > 0) {
S
Siying Dong 已提交
1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145
    bg_cv_.Wait();
  }
  bg_work_paused_++;
  return Status::OK();
}

Status DBImpl::ContinueBackgroundWork() {
  InstrumentedMutexLock guard_lock(&mutex_);
  if (bg_work_paused_ == 0) {
    return Status::InvalidArgument();
  }
  assert(bg_work_paused_ > 0);
  assert(bg_compaction_paused_ > 0);
  bg_compaction_paused_--;
  bg_work_paused_--;
  // It's sufficient to check just bg_work_paused_ here since
  // bg_work_paused_ is always no greater than bg_compaction_paused_
  if (bg_work_paused_ == 0) {
    MaybeScheduleFlushOrCompaction();
  }
  return Status::OK();
}

1146 1147
void DBImpl::NotifyOnCompactionBegin(ColumnFamilyData* cfd, Compaction* c,
                                     const Status& st,
P
Peter Pei 已提交
1148 1149 1150 1151 1152 1153 1154 1155 1156 1157
                                     const CompactionJobStats& job_stats,
                                     int job_id) {
#ifndef ROCKSDB_LITE
  if (immutable_db_options_.listeners.empty()) {
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
1158 1159 1160 1161
  if (c->is_manual_compaction() &&
      manual_compaction_paused_.load(std::memory_order_acquire)) {
    return;
  }
P
Peter Pei 已提交
1162 1163 1164 1165 1166 1167
  Version* current = cfd->current();
  current->Ref();
  // release lock while notifying events
  mutex_.Unlock();
  TEST_SYNC_POINT("DBImpl::NotifyOnCompactionBegin::UnlockMutex");
  {
1168
    CompactionJobInfo info{};
P
Peter Pei 已提交
1169 1170 1171 1172 1173 1174 1175 1176 1177 1178 1179 1180
    info.cf_name = cfd->GetName();
    info.status = st;
    info.thread_id = env_->GetThreadID();
    info.job_id = job_id;
    info.base_input_level = c->start_level();
    info.output_level = c->output_level();
    info.stats = job_stats;
    info.table_properties = c->GetOutputTableProperties();
    info.compaction_reason = c->compaction_reason();
    info.compression = c->output_compression();
    for (size_t i = 0; i < c->num_input_levels(); ++i) {
      for (const auto fmd : *c->inputs(i)) {
1181 1182
        const FileDescriptor& desc = fmd->fd;
        const uint64_t file_number = desc.GetNumber();
P
Peter Pei 已提交
1183
        auto fn = TableFileName(c->immutable_cf_options()->cf_paths,
1184
                                file_number, desc.GetPathId());
P
Peter Pei 已提交
1185
        info.input_files.push_back(fn);
1186 1187
        info.input_file_infos.push_back(CompactionFileInfo{
            static_cast<int>(i), file_number, fmd->oldest_blob_file_number});
P
Peter Pei 已提交
1188 1189 1190 1191 1192 1193 1194 1195 1196
        if (info.table_properties.count(fn) == 0) {
          std::shared_ptr<const TableProperties> tp;
          auto s = current->GetTableProperties(&tp, fmd, &fn);
          if (s.ok()) {
            info.table_properties[fn] = tp;
          }
        }
      }
    }
1197
    for (const auto& newf : c->edit()->GetNewFiles()) {
1198 1199 1200
      const FileMetaData& meta = newf.second;
      const FileDescriptor& desc = meta.fd;
      const uint64_t file_number = desc.GetNumber();
P
Peter Pei 已提交
1201
      info.output_files.push_back(TableFileName(
1202 1203 1204
          c->immutable_cf_options()->cf_paths, file_number, desc.GetPathId()));
      info.output_file_infos.push_back(CompactionFileInfo{
          newf.first, file_number, meta.oldest_blob_file_number});
P
Peter Pei 已提交
1205 1206 1207 1208 1209 1210 1211 1212 1213 1214 1215 1216 1217 1218 1219 1220
    }
    for (auto listener : immutable_db_options_.listeners) {
      listener->OnCompactionBegin(this, info);
    }
  }
  mutex_.Lock();
  current->Unref();
#else
  (void)cfd;
  (void)c;
  (void)st;
  (void)job_stats;
  (void)job_id;
#endif  // ROCKSDB_LITE
}

S
Siying Dong 已提交
1221
void DBImpl::NotifyOnCompactionCompleted(
1222 1223
    ColumnFamilyData* cfd, Compaction* c, const Status& st,
    const CompactionJobStats& compaction_job_stats, const int job_id) {
S
Siying Dong 已提交
1224 1225 1226 1227 1228 1229 1230 1231
#ifndef ROCKSDB_LITE
  if (immutable_db_options_.listeners.size() == 0U) {
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
1232 1233 1234 1235
  if (c->is_manual_compaction() &&
      manual_compaction_paused_.load(std::memory_order_acquire)) {
    return;
  }
1236 1237
  Version* current = cfd->current();
  current->Ref();
S
Siying Dong 已提交
1238 1239 1240 1241
  // release lock while notifying events
  mutex_.Unlock();
  TEST_SYNC_POINT("DBImpl::NotifyOnCompactionCompleted::UnlockMutex");
  {
1242
    CompactionJobInfo info{};
1243 1244
    BuildCompactionJobInfo(cfd, c, st, compaction_job_stats, job_id, current,
                           &info);
S
Siying Dong 已提交
1245 1246 1247 1248 1249
    for (auto listener : immutable_db_options_.listeners) {
      listener->OnCompactionCompleted(this, info);
    }
  }
  mutex_.Lock();
1250
  current->Unref();
S
Siying Dong 已提交
1251 1252
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
1253 1254 1255 1256 1257 1258
#else
  (void)cfd;
  (void)c;
  (void)st;
  (void)compaction_job_stats;
  (void)job_id;
S
Siying Dong 已提交
1259 1260 1261 1262 1263 1264 1265 1266 1267 1268 1269
#endif  // ROCKSDB_LITE
}

// REQUIREMENT: block all background work by calling PauseBackgroundWork()
// before calling this function
Status DBImpl::ReFitLevel(ColumnFamilyData* cfd, int level, int target_level) {
  assert(level < cfd->NumberLevels());
  if (target_level >= cfd->NumberLevels()) {
    return Status::InvalidArgument("Target level exceeds number of levels");
  }

1270
  SuperVersionContext sv_context(/* create_superversion */ true);
S
Siying Dong 已提交
1271 1272 1273 1274 1275 1276 1277 1278 1279 1280 1281 1282 1283 1284 1285 1286 1287 1288 1289 1290 1291 1292 1293 1294 1295 1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311 1312 1313 1314 1315

  Status status;

  InstrumentedMutexLock guard_lock(&mutex_);

  // only allow one thread refitting
  if (refitting_level_) {
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "[ReFitLevel] another thread is refitting");
    return Status::NotSupported("another thread is refitting");
  }
  refitting_level_ = true;

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

  auto* vstorage = cfd->current()->storage_info();
  if (to_level > level) {
    if (level == 0) {
      return Status::NotSupported(
          "Cannot change from level 0 to other levels.");
    }
    // Check levels are empty for a trivial move
    for (int l = level + 1; l <= to_level; l++) {
      if (vstorage->NumLevelFiles(l) > 0) {
        return Status::NotSupported(
            "Levels between source and target are not empty for a move.");
      }
    }
  }
  if (to_level != level) {
    ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
                    "[%s] Before refitting:\n%s", cfd->GetName().c_str(),
                    cfd->current()->DebugString().data());

    VersionEdit edit;
    edit.SetColumnFamily(cfd->GetID());
    for (const auto& f : vstorage->LevelFiles(level)) {
      edit.DeleteFile(level, f->fd.GetNumber());
      edit.AddFile(to_level, f->fd.GetNumber(), f->fd.GetPathId(),
                   f->fd.GetFileSize(), f->smallest, f->largest,
1316
                   f->fd.smallest_seqno, f->fd.largest_seqno,
1317
                   f->marked_for_compaction, f->oldest_blob_file_number,
1318 1319
                   f->oldest_ancester_time, f->file_creation_time,
                   f->file_checksum, f->file_checksum_func_name);
S
Siying Dong 已提交
1320 1321 1322 1323 1324 1325 1326
    }
    ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
                    "[%s] Apply version edit:\n%s", cfd->GetName().c_str(),
                    edit.DebugString().data());

    status = versions_->LogAndApply(cfd, mutable_cf_options, &edit, &mutex_,
                                    directories_.GetDbDir());
1327
    InstallSuperVersionAndScheduleWork(cfd, &sv_context, mutable_cf_options);
S
Siying Dong 已提交
1328 1329 1330 1331 1332 1333 1334 1335 1336 1337 1338

    ROCKS_LOG_DEBUG(immutable_db_options_.info_log, "[%s] LogAndApply: %s\n",
                    cfd->GetName().c_str(), status.ToString().data());

    if (status.ok()) {
      ROCKS_LOG_DEBUG(immutable_db_options_.info_log,
                      "[%s] After refitting:\n%s", cfd->GetName().c_str(),
                      cfd->current()->DebugString().data());
    }
  }

1339
  sv_context.Clean();
S
Siying Dong 已提交
1340 1341 1342 1343 1344 1345 1346 1347 1348 1349
  refitting_level_ = false;

  return status;
}

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

A
Andrew Kryczka 已提交
1350
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* /*column_family*/) {
S
Siying Dong 已提交
1351 1352 1353 1354 1355 1356
  return 0;
}

int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  InstrumentedMutexLock l(&mutex_);
1357 1358 1359
  return cfh->cfd()
      ->GetSuperVersion()
      ->mutable_cf_options.level0_stop_writes_trigger;
S
Siying Dong 已提交
1360 1361 1362 1363 1364
}

Status DBImpl::Flush(const FlushOptions& flush_options,
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1365 1366
  ROCKS_LOG_INFO(immutable_db_options_.info_log, "[%s] Manual flush start.",
                 cfh->GetName().c_str());
Y
Yanqin Jin 已提交
1367
  Status s;
1368
  if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1369 1370 1371 1372 1373 1374
    s = AtomicFlushMemTables({cfh->cfd()}, flush_options,
                             FlushReason::kManualFlush);
  } else {
    s = FlushMemTable(cfh->cfd(), flush_options, FlushReason::kManualFlush);
  }

1375 1376 1377 1378
  ROCKS_LOG_INFO(immutable_db_options_.info_log,
                 "[%s] Manual flush finished, status: %s\n",
                 cfh->GetName().c_str(), s.ToString().c_str());
  return s;
S
Siying Dong 已提交
1379 1380
}

Y
Yanqin Jin 已提交
1381 1382
Status DBImpl::Flush(const FlushOptions& flush_options,
                     const std::vector<ColumnFamilyHandle*>& column_families) {
1383
  Status s;
1384
  if (!immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1385 1386 1387 1388 1389
    for (auto cfh : column_families) {
      s = Flush(flush_options, cfh);
      if (!s.ok()) {
        break;
      }
1390
    }
Y
Yanqin Jin 已提交
1391 1392 1393 1394 1395 1396 1397 1398
  } else {
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "Manual atomic flush start.\n"
                   "=====Column families:=====");
    for (auto cfh : column_families) {
      auto cfhi = static_cast<ColumnFamilyHandleImpl*>(cfh);
      ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s",
                     cfhi->GetName().c_str());
1399
    }
Y
Yanqin Jin 已提交
1400 1401 1402 1403 1404 1405 1406 1407 1408 1409
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "=====End of column families list=====");
    autovector<ColumnFamilyData*> cfds;
    std::for_each(column_families.begin(), column_families.end(),
                  [&cfds](ColumnFamilyHandle* elem) {
                    auto cfh = static_cast<ColumnFamilyHandleImpl*>(elem);
                    cfds.emplace_back(cfh->cfd());
                  });
    s = AtomicFlushMemTables(cfds, flush_options, FlushReason::kManualFlush);
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
1410 1411 1412
                   "Manual atomic flush finished, status: %s\n"
                   "=====Column families:=====",
                   s.ToString().c_str());
Y
Yanqin Jin 已提交
1413 1414 1415 1416
    for (auto cfh : column_families) {
      auto cfhi = static_cast<ColumnFamilyHandleImpl*>(cfh);
      ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s",
                     cfhi->GetName().c_str());
1417
    }
Y
Yanqin Jin 已提交
1418 1419
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "=====End of column families list=====");
1420 1421 1422 1423
  }
  return s;
}

1424 1425 1426 1427 1428
Status DBImpl::RunManualCompaction(
    ColumnFamilyData* cfd, int input_level, int output_level,
    const CompactRangeOptions& compact_range_options, const Slice* begin,
    const Slice* end, bool exclusive, bool disallow_trivial_move,
    uint64_t max_file_num_to_ignore) {
S
Siying Dong 已提交
1429 1430 1431 1432 1433 1434 1435 1436
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);

  InternalKey begin_storage, end_storage;
  CompactionArg* ca;

  bool scheduled = false;
  bool manual_conflict = false;
1437
  ManualCompactionState manual;
S
Siying Dong 已提交
1438 1439 1440
  manual.cfd = cfd;
  manual.input_level = input_level;
  manual.output_level = output_level;
1441
  manual.output_path_id = compact_range_options.target_path_id;
S
Siying Dong 已提交
1442 1443 1444 1445 1446 1447 1448 1449 1450 1451 1452 1453
  manual.done = false;
  manual.in_progress = false;
  manual.incomplete = false;
  manual.exclusive = exclusive;
  manual.disallow_trivial_move = disallow_trivial_move;
  // For universal compaction, we enforce every manual compaction to compact
  // all files.
  if (begin == nullptr ||
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
    manual.begin = nullptr;
  } else {
A
Amy Xu 已提交
1454
    begin_storage.SetMinPossibleForUserKey(*begin);
S
Siying Dong 已提交
1455 1456 1457 1458 1459 1460 1461
    manual.begin = &begin_storage;
  }
  if (end == nullptr ||
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
    manual.end = nullptr;
  } else {
A
Amy Xu 已提交
1462
    end_storage.SetMaxPossibleForUserKey(*end);
S
Siying Dong 已提交
1463 1464 1465 1466 1467 1468 1469 1470 1471 1472 1473 1474 1475 1476 1477 1478 1479 1480 1481 1482 1483 1484 1485 1486
    manual.end = &end_storage;
  }

  TEST_SYNC_POINT("DBImpl::RunManualCompaction:0");
  TEST_SYNC_POINT("DBImpl::RunManualCompaction:1");
  InstrumentedMutexLock l(&mutex_);

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

  AddManualCompaction(&manual);
  TEST_SYNC_POINT_CALLBACK("DBImpl::RunManualCompaction:NotScheduled", &mutex_);
  if (exclusive) {
1487 1488
    while (bg_bottom_compaction_scheduled_ > 0 ||
           bg_compaction_scheduled_ > 0) {
1489
      TEST_SYNC_POINT("DBImpl::RunManualCompaction:WaitScheduled");
S
Siying Dong 已提交
1490 1491 1492 1493 1494 1495 1496 1497 1498 1499 1500 1501
      ROCKS_LOG_INFO(
          immutable_db_options_.info_log,
          "[%s] Manual compaction waiting for all other scheduled background "
          "compactions to finish",
          cfd->GetName().c_str());
      bg_cv_.Wait();
    }
  }

  ROCKS_LOG_INFO(immutable_db_options_.info_log,
                 "[%s] Manual compaction starting", cfd->GetName().c_str());

1502 1503
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
                       immutable_db_options_.info_log.get());
S
Siying Dong 已提交
1504 1505 1506 1507 1508 1509
  // We don't check bg_error_ here, because if we get the error in compaction,
  // the compaction will set manual.status to bg_error_ and set manual.done to
  // true.
  while (!manual.done) {
    assert(HasPendingManualCompaction());
    manual_conflict = false;
1510
    Compaction* compaction = nullptr;
S
Siying Dong 已提交
1511 1512
    if (ShouldntRunManualCompaction(&manual) || (manual.in_progress == true) ||
        scheduled ||
1513
        (((manual.manual_end = &manual.tmp_storage1) != nullptr) &&
1514 1515
         ((compaction = manual.cfd->CompactRange(
               *manual.cfd->GetLatestMutableCFOptions(), manual.input_level,
1516 1517 1518
               manual.output_level, compact_range_options, manual.begin,
               manual.end, &manual.manual_end, &manual_conflict,
               max_file_num_to_ignore)) == nullptr &&
1519
          manual_conflict))) {
S
Siying Dong 已提交
1520 1521 1522 1523 1524 1525 1526 1527 1528 1529 1530
      // exclusive manual compactions should not see a conflict during
      // CompactRange
      assert(!exclusive || !manual_conflict);
      // Running either this or some other manual compaction
      bg_cv_.Wait();
      if (scheduled && manual.incomplete == true) {
        assert(!manual.in_progress);
        scheduled = false;
        manual.incomplete = false;
      }
    } else if (!scheduled) {
1531
      if (compaction == nullptr) {
S
Siying Dong 已提交
1532 1533 1534 1535 1536 1537
        manual.done = true;
        bg_cv_.SignalAll();
        continue;
      }
      ca = new CompactionArg;
      ca->db = this;
1538 1539 1540
      ca->prepicked_compaction = new PrepickedCompaction;
      ca->prepicked_compaction->manual_compaction_state = &manual;
      ca->prepicked_compaction->compaction = compaction;
Y
Yanqin Jin 已提交
1541 1542 1543 1544
      if (!RequestCompactionToken(
              cfd, true, &ca->prepicked_compaction->task_token, &log_buffer)) {
        // Don't throttle manual compaction, only count outstanding tasks.
        assert(false);
1545
      }
S
Siying Dong 已提交
1546 1547
      manual.incomplete = false;
      bg_compaction_scheduled_++;
1548 1549 1550 1551 1552 1553
      Env::Priority thread_pool_pri = Env::Priority::LOW;
      if (compaction->bottommost_level() &&
          env_->GetBackgroundThreads(Env::Priority::BOTTOM) > 0) {
        thread_pool_pri = Env::Priority::BOTTOM;
      }
      env_->Schedule(&DBImpl::BGWorkCompaction, ca, thread_pool_pri, this,
1554
                     &DBImpl::UnscheduleCompactionCallback);
S
Siying Dong 已提交
1555 1556 1557 1558
      scheduled = true;
    }
  }

1559
  log_buffer.FlushBufferToLog();
S
Siying Dong 已提交
1560 1561 1562 1563 1564 1565 1566
  assert(!manual.in_progress);
  assert(HasPendingManualCompaction());
  RemoveManualCompaction(&manual);
  bg_cv_.SignalAll();
  return manual.status;
}

Y
Yanqin Jin 已提交
1567 1568 1569
void DBImpl::GenerateFlushRequest(const autovector<ColumnFamilyData*>& cfds,
                                  FlushRequest* req) {
  assert(req != nullptr);
1570
  req->reserve(cfds.size());
Y
Yanqin Jin 已提交
1571 1572 1573 1574 1575 1576 1577 1578 1579 1580
  for (const auto cfd : cfds) {
    if (nullptr == cfd) {
      // cfd may be null, see DBImpl::ScheduleFlushes
      continue;
    }
    uint64_t max_memtable_id = cfd->imm()->GetLatestMemTableID();
    req->emplace_back(cfd, max_memtable_id);
  }
}

S
Siying Dong 已提交
1581 1582
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
                             const FlushOptions& flush_options,
1583
                             FlushReason flush_reason, bool writes_stopped) {
S
Siying Dong 已提交
1584
  Status s;
1585
  uint64_t flush_memtable_id = 0;
1586 1587 1588 1589 1590 1591 1592 1593
  if (!flush_options.allow_write_stall) {
    bool flush_needed = true;
    s = WaitUntilFlushWouldNotStallWrites(cfd, &flush_needed);
    TEST_SYNC_POINT("DBImpl::FlushMemTable:StallWaitDone");
    if (!s.ok() || !flush_needed) {
      return s;
    }
  }
1594
  FlushRequest flush_req;
S
Siying Dong 已提交
1595 1596 1597 1598 1599
  {
    WriteContext context;
    InstrumentedMutexLock guard_lock(&mutex_);

    WriteThread::Writer w;
1600
    WriteThread::Writer nonmem_w;
S
Siying Dong 已提交
1601 1602
    if (!writes_stopped) {
      write_thread_.EnterUnbatched(&w, &mutex_);
1603 1604 1605
      if (two_write_queues_) {
        nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_);
      }
S
Siying Dong 已提交
1606
    }
1607
    WaitForPendingWrites();
S
Siying Dong 已提交
1608

1609
    if (!cfd->mem()->IsEmpty() || !cached_recoverable_state_empty_.load()) {
1610
      s = SwitchMemtable(cfd, &context);
1611 1612 1613 1614 1615 1616 1617
    }
    if (s.ok()) {
      if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() ||
          !cached_recoverable_state_empty_.load()) {
        flush_memtable_id = cfd->imm()->GetLatestMemTableID();
        flush_req.emplace_back(cfd, flush_memtable_id);
      }
1618 1619 1620 1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633 1634 1635 1636 1637
      if (immutable_db_options_.persist_stats_to_disk) {
        ColumnFamilyData* cfd_stats =
            versions_->GetColumnFamilySet()->GetColumnFamily(
                kPersistentStatsColumnFamilyName);
        if (cfd_stats != nullptr && cfd_stats != cfd &&
            !cfd_stats->mem()->IsEmpty()) {
          // only force flush stats CF when it will be the only CF lagging
          // behind after the current flush
          bool stats_cf_flush_needed = true;
          for (auto* loop_cfd : *versions_->GetColumnFamilySet()) {
            if (loop_cfd == cfd_stats || loop_cfd == cfd) {
              continue;
            }
            if (loop_cfd->GetLogNumber() <= cfd_stats->GetLogNumber()) {
              stats_cf_flush_needed = false;
            }
          }
          if (stats_cf_flush_needed) {
            ROCKS_LOG_INFO(immutable_db_options_.info_log,
                           "Force flushing stats CF with manual flush of %s "
1638 1639
                           "to avoid holding old logs",
                           cfd->GetName().c_str());
1640 1641 1642 1643 1644 1645
            s = SwitchMemtable(cfd_stats, &context);
            flush_memtable_id = cfd_stats->imm()->GetLatestMemTableID();
            flush_req.emplace_back(cfd_stats, flush_memtable_id);
          }
        }
      }
1646 1647 1648 1649 1650 1651 1652
    }

    if (s.ok() && !flush_req.empty()) {
      for (auto& elem : flush_req) {
        ColumnFamilyData* loop_cfd = elem.first;
        loop_cfd->imm()->FlushRequested();
      }
1653 1654 1655 1656 1657 1658 1659 1660 1661 1662
      // If the caller wants to wait for this flush to complete, it indicates
      // that the caller expects the ColumnFamilyData not to be free'ed by
      // other threads which may drop the column family concurrently.
      // Therefore, we increase the cfd's ref count.
      if (flush_options.wait) {
        for (auto& elem : flush_req) {
          ColumnFamilyData* loop_cfd = elem.first;
          loop_cfd->Ref();
        }
      }
1663 1664 1665
      SchedulePendingFlush(flush_req, flush_reason);
      MaybeScheduleFlushOrCompaction();
    }
S
Siying Dong 已提交
1666 1667 1668

    if (!writes_stopped) {
      write_thread_.ExitUnbatched(&w);
1669 1670 1671
      if (two_write_queues_) {
        nonmem_write_thread_.ExitUnbatched(&nonmem_w);
      }
S
Siying Dong 已提交
1672 1673
    }
  }
1674 1675
  TEST_SYNC_POINT("DBImpl::FlushMemTable:AfterScheduleFlush");
  TEST_SYNC_POINT("DBImpl::FlushMemTable:BeforeWaitForBgFlush");
S
Siying Dong 已提交
1676
  if (s.ok() && flush_options.wait) {
1677 1678 1679 1680 1681 1682
    autovector<ColumnFamilyData*> cfds;
    autovector<const uint64_t*> flush_memtable_ids;
    for (auto& iter : flush_req) {
      cfds.push_back(iter.first);
      flush_memtable_ids.push_back(&(iter.second));
    }
Y
Yanqin Jin 已提交
1683 1684
    s = WaitForFlushMemTables(cfds, flush_memtable_ids,
                              (flush_reason == FlushReason::kErrorRecovery));
1685
    InstrumentedMutexLock lock_guard(&mutex_);
1686
    for (auto* tmp_cfd : cfds) {
1687
      tmp_cfd->UnrefAndTryDelete();
1688
    }
S
Siying Dong 已提交
1689
  }
1690
  TEST_SYNC_POINT("DBImpl::FlushMemTable:FlushMemTableFinished");
S
Siying Dong 已提交
1691 1692 1693
  return s;
}

1694
// Flush all elements in 'column_family_datas'
Y
Yanqin Jin 已提交
1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708 1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722
// and atomically record the result to the MANIFEST.
Status DBImpl::AtomicFlushMemTables(
    const autovector<ColumnFamilyData*>& column_family_datas,
    const FlushOptions& flush_options, FlushReason flush_reason,
    bool writes_stopped) {
  Status s;
  if (!flush_options.allow_write_stall) {
    int num_cfs_to_flush = 0;
    for (auto cfd : column_family_datas) {
      bool flush_needed = true;
      s = WaitUntilFlushWouldNotStallWrites(cfd, &flush_needed);
      if (!s.ok()) {
        return s;
      } else if (flush_needed) {
        ++num_cfs_to_flush;
      }
    }
    if (0 == num_cfs_to_flush) {
      return s;
    }
  }
  FlushRequest flush_req;
  autovector<ColumnFamilyData*> cfds;
  {
    WriteContext context;
    InstrumentedMutexLock guard_lock(&mutex_);

    WriteThread::Writer w;
1723
    WriteThread::Writer nonmem_w;
Y
Yanqin Jin 已提交
1724 1725
    if (!writes_stopped) {
      write_thread_.EnterUnbatched(&w, &mutex_);
1726 1727 1728
      if (two_write_queues_) {
        nonmem_write_thread_.EnterUnbatched(&nonmem_w, &mutex_);
      }
Y
Yanqin Jin 已提交
1729
    }
1730
    WaitForPendingWrites();
Y
Yanqin Jin 已提交
1731 1732 1733 1734 1735 1736 1737 1738 1739 1740 1741

    for (auto cfd : column_family_datas) {
      if (cfd->IsDropped()) {
        continue;
      }
      if (cfd->imm()->NumNotFlushed() != 0 || !cfd->mem()->IsEmpty() ||
          !cached_recoverable_state_empty_.load()) {
        cfds.emplace_back(cfd);
      }
    }
    for (auto cfd : cfds) {
1742 1743 1744
      if (cfd->mem()->IsEmpty() && cached_recoverable_state_empty_.load()) {
        continue;
      }
Y
Yanqin Jin 已提交
1745 1746
      cfd->Ref();
      s = SwitchMemtable(cfd, &context);
1747
      cfd->UnrefAndTryDelete();
Y
Yanqin Jin 已提交
1748 1749 1750 1751 1752 1753 1754 1755 1756
      if (!s.ok()) {
        break;
      }
    }
    if (s.ok()) {
      AssignAtomicFlushSeq(cfds);
      for (auto cfd : cfds) {
        cfd->imm()->FlushRequested();
      }
1757 1758 1759 1760 1761 1762 1763 1764 1765
      // If the caller wants to wait for this flush to complete, it indicates
      // that the caller expects the ColumnFamilyData not to be free'ed by
      // other threads which may drop the column family concurrently.
      // Therefore, we increase the cfd's ref count.
      if (flush_options.wait) {
        for (auto cfd : cfds) {
          cfd->Ref();
        }
      }
Y
Yanqin Jin 已提交
1766 1767 1768 1769 1770 1771 1772
      GenerateFlushRequest(cfds, &flush_req);
      SchedulePendingFlush(flush_req, flush_reason);
      MaybeScheduleFlushOrCompaction();
    }

    if (!writes_stopped) {
      write_thread_.ExitUnbatched(&w);
1773 1774 1775
      if (two_write_queues_) {
        nonmem_write_thread_.ExitUnbatched(&nonmem_w);
      }
Y
Yanqin Jin 已提交
1776 1777
    }
  }
1778
  TEST_SYNC_POINT("DBImpl::AtomicFlushMemTables:AfterScheduleFlush");
1779
  TEST_SYNC_POINT("DBImpl::AtomicFlushMemTables:BeforeWaitForBgFlush");
Y
Yanqin Jin 已提交
1780 1781 1782 1783 1784 1785 1786
  if (s.ok() && flush_options.wait) {
    autovector<const uint64_t*> flush_memtable_ids;
    for (auto& iter : flush_req) {
      flush_memtable_ids.push_back(&(iter.second));
    }
    s = WaitForFlushMemTables(cfds, flush_memtable_ids,
                              (flush_reason == FlushReason::kErrorRecovery));
1787
    InstrumentedMutexLock lock_guard(&mutex_);
1788
    for (auto* cfd : cfds) {
1789
      cfd->UnrefAndTryDelete();
1790
    }
Y
Yanqin Jin 已提交
1791 1792 1793 1794
  }
  return s;
}

1795 1796 1797 1798 1799 1800 1801
// Calling FlushMemTable(), whether from DB::Flush() or from Backup Engine, can
// cause write stall, for example if one memtable is being flushed already.
// This method tries to avoid write stall (similar to CompactRange() behavior)
// it emulates how the SuperVersion / LSM would change if flush happens, checks
// it against various constrains and delays flush if it'd cause write stall.
// Called should check status and flush_needed to see if flush already happened.
Status DBImpl::WaitUntilFlushWouldNotStallWrites(ColumnFamilyData* cfd,
1802
                                                 bool* flush_needed) {
1803 1804 1805 1806 1807 1808 1809
  {
    *flush_needed = true;
    InstrumentedMutexLock l(&mutex_);
    uint64_t orig_active_memtable_id = cfd->mem()->GetID();
    WriteStallCondition write_stall_condition = WriteStallCondition::kNormal;
    do {
      if (write_stall_condition != WriteStallCondition::kNormal) {
1810 1811 1812 1813 1814 1815 1816 1817
        // Same error handling as user writes: Don't wait if there's a
        // background error, even if it's a soft error. We might wait here
        // indefinitely as the pending flushes/compactions may never finish
        // successfully, resulting in the stall condition lasting indefinitely
        if (error_handler_.IsBGWorkStopped()) {
          return error_handler_.GetBGError();
        }

1818 1819 1820 1821 1822 1823 1824
        TEST_SYNC_POINT("DBImpl::WaitUntilFlushWouldNotStallWrites:StallWait");
        ROCKS_LOG_INFO(immutable_db_options_.info_log,
                       "[%s] WaitUntilFlushWouldNotStallWrites"
                       " waiting on stall conditions to clear",
                       cfd->GetName().c_str());
        bg_cv_.Wait();
      }
1825 1826 1827 1828
      if (cfd->IsDropped()) {
        return Status::ColumnFamilyDropped();
      }
      if (shutting_down_.load(std::memory_order_acquire)) {
1829 1830 1831 1832 1833 1834 1835 1836 1837 1838 1839 1840 1841 1842 1843 1844 1845 1846 1847 1848 1849 1850 1851 1852 1853 1854 1855 1856 1857 1858 1859 1860 1861 1862 1863 1864 1865 1866 1867 1868
        return Status::ShutdownInProgress();
      }

      uint64_t earliest_memtable_id =
          std::min(cfd->mem()->GetID(), cfd->imm()->GetEarliestMemTableID());
      if (earliest_memtable_id > orig_active_memtable_id) {
        // We waited so long that the memtable we were originally waiting on was
        // flushed.
        *flush_needed = false;
        return Status::OK();
      }

      const auto& mutable_cf_options = *cfd->GetLatestMutableCFOptions();
      const auto* vstorage = cfd->current()->storage_info();

      // Skip stalling check if we're below auto-flush and auto-compaction
      // triggers. If it stalled in these conditions, that'd mean the stall
      // triggers are so low that stalling is needed for any background work. In
      // that case we shouldn't wait since background work won't be scheduled.
      if (cfd->imm()->NumNotFlushed() <
              cfd->ioptions()->min_write_buffer_number_to_merge &&
          vstorage->l0_delay_trigger_count() <
              mutable_cf_options.level0_file_num_compaction_trigger) {
        break;
      }

      // check whether one extra immutable memtable or an extra L0 file would
      // cause write stalling mode to be entered. It could still enter stall
      // mode due to pending compaction bytes, but that's less common
      write_stall_condition =
          ColumnFamilyData::GetWriteStallConditionAndCause(
              cfd->imm()->NumNotFlushed() + 1,
              vstorage->l0_delay_trigger_count() + 1,
              vstorage->estimated_compaction_needed_bytes(), mutable_cf_options)
              .first;
    } while (write_stall_condition != WriteStallCondition::kNormal);
  }
  return Status::OK();
}

1869 1870 1871 1872 1873 1874 1875 1876
// Wait for memtables to be flushed for multiple column families.
// let N = cfds.size()
// for i in [0, N),
//  1) if flush_memtable_ids[i] is not null, then the memtables with lower IDs
//     have to be flushed for THIS column family;
//  2) if flush_memtable_ids[i] is null, then all memtables in THIS column
//     family have to be flushed.
// Finish waiting when ALL column families finish flushing memtables.
Y
Yanqin Jin 已提交
1877 1878
// resuming_from_bg_err indicates whether the caller is trying to resume from
// background error or in normal processing.
1879 1880
Status DBImpl::WaitForFlushMemTables(
    const autovector<ColumnFamilyData*>& cfds,
Y
Yanqin Jin 已提交
1881 1882
    const autovector<const uint64_t*>& flush_memtable_ids,
    bool resuming_from_bg_err) {
1883
  int num = static_cast<int>(cfds.size());
S
Siying Dong 已提交
1884 1885
  // Wait until the compaction completes
  InstrumentedMutexLock l(&mutex_);
Y
Yanqin Jin 已提交
1886 1887 1888
  // If the caller is trying to resume from bg error, then
  // error_handler_.IsDBStopped() is true.
  while (resuming_from_bg_err || !error_handler_.IsDBStopped()) {
S
Siying Dong 已提交
1889 1890 1891
    if (shutting_down_.load(std::memory_order_acquire)) {
      return Status::ShutdownInProgress();
    }
Y
Yanqin Jin 已提交
1892 1893 1894 1895
    // If an error has occurred during resumption, then no need to wait.
    if (!error_handler_.GetRecoveryError().ok()) {
      break;
    }
1896 1897 1898 1899 1900 1901 1902 1903 1904 1905 1906 1907 1908 1909 1910
    // Number of column families that have been dropped.
    int num_dropped = 0;
    // Number of column families that have finished flush.
    int num_finished = 0;
    for (int i = 0; i < num; ++i) {
      if (cfds[i]->IsDropped()) {
        ++num_dropped;
      } else if (cfds[i]->imm()->NumNotFlushed() == 0 ||
                 (flush_memtable_ids[i] != nullptr &&
                  cfds[i]->imm()->GetEarliestMemTableID() >
                      *flush_memtable_ids[i])) {
        ++num_finished;
      }
    }
    if (1 == num_dropped && 1 == num) {
1911
      return Status::ColumnFamilyDropped();
S
Siying Dong 已提交
1912
    }
1913 1914 1915 1916 1917
    // Column families involved in this flush request have either been dropped
    // or finished flush. Then it's time to finish waiting.
    if (num_dropped + num_finished == num) {
      break;
    }
S
Siying Dong 已提交
1918 1919
    bg_cv_.Wait();
  }
1920
  Status s;
Y
Yanqin Jin 已提交
1921 1922 1923
  // If not resuming from bg error, and an error has caused the DB to stop,
  // then report the bg error to caller.
  if (!resuming_from_bg_err && error_handler_.IsDBStopped()) {
1924
    s = error_handler_.GetBGError();
S
Siying Dong 已提交
1925 1926 1927 1928 1929 1930 1931 1932 1933 1934 1935 1936 1937 1938 1939 1940 1941 1942
  }
  return s;
}

Status DBImpl::EnableAutoCompaction(
    const std::vector<ColumnFamilyHandle*>& column_family_handles) {
  Status s;
  for (auto cf_ptr : column_family_handles) {
    Status status =
        this->SetOptions(cf_ptr, {{"disable_auto_compactions", "false"}});
    if (!status.ok()) {
      s = status;
    }
  }

  return s;
}

1943 1944 1945 1946 1947 1948 1949 1950
void DBImpl::DisableManualCompaction() {
  manual_compaction_paused_.store(true, std::memory_order_release);
}

void DBImpl::EnableManualCompaction() {
  manual_compaction_paused_.store(false, std::memory_order_release);
}

S
Siying Dong 已提交
1951 1952 1953 1954 1955 1956 1957 1958 1959
void DBImpl::MaybeScheduleFlushOrCompaction() {
  mutex_.AssertHeld();
  if (!opened_successfully_) {
    // Compaction may introduce data race to DB open
    return;
  }
  if (bg_work_paused_ > 0) {
    // we paused the background work
    return;
1960
  } else if (error_handler_.IsBGWorkStopped() &&
1961
             !error_handler_.IsRecoveryInProgress()) {
1962 1963 1964 1965
    // There has been a hard error and this call is not part of the recovery
    // sequence. Bail out here so we don't get into an endless loop of
    // scheduling BG work which will again call this function
    return;
S
Siying Dong 已提交
1966 1967 1968 1969
  } else if (shutting_down_.load(std::memory_order_acquire)) {
    // DB is being deleted; no more background compactions
    return;
  }
1970
  auto bg_job_limits = GetBGJobLimits();
1971
  bool is_flush_pool_empty =
1972
      env_->GetBackgroundThreads(Env::Priority::HIGH) == 0;
1973
  while (!is_flush_pool_empty && unscheduled_flushes_ > 0 &&
1974
         bg_flush_scheduled_ < bg_job_limits.max_flushes) {
S
Siying Dong 已提交
1975
    bg_flush_scheduled_++;
1976 1977 1978 1979 1980
    FlushThreadArg* fta = new FlushThreadArg;
    fta->db_ = this;
    fta->thread_pri_ = Env::Priority::HIGH;
    env_->Schedule(&DBImpl::BGWorkFlush, fta, Env::Priority::HIGH, this,
                   &DBImpl::UnscheduleFlushCallback);
1981 1982 1983 1984
    --unscheduled_flushes_;
    TEST_SYNC_POINT_CALLBACK(
        "DBImpl::MaybeScheduleFlushOrCompaction:AfterSchedule:0",
        &unscheduled_flushes_);
S
Siying Dong 已提交
1985 1986
  }

1987 1988 1989
  // special case -- if high-pri (flush) thread pool is empty, then schedule
  // flushes in low-pri (compaction) thread pool.
  if (is_flush_pool_empty) {
S
Siying Dong 已提交
1990 1991
    while (unscheduled_flushes_ > 0 &&
           bg_flush_scheduled_ + bg_compaction_scheduled_ <
1992
               bg_job_limits.max_flushes) {
S
Siying Dong 已提交
1993
      bg_flush_scheduled_++;
1994 1995 1996 1997 1998
      FlushThreadArg* fta = new FlushThreadArg;
      fta->db_ = this;
      fta->thread_pri_ = Env::Priority::LOW;
      env_->Schedule(&DBImpl::BGWorkFlush, fta, Env::Priority::LOW, this,
                     &DBImpl::UnscheduleFlushCallback);
1999
      --unscheduled_flushes_;
S
Siying Dong 已提交
2000 2001 2002 2003 2004 2005
    }
  }

  if (bg_compaction_paused_ > 0) {
    // we paused the background compaction
    return;
2006 2007 2008 2009 2010 2011
  } else if (error_handler_.IsBGWorkStopped()) {
    // Compaction is not part of the recovery sequence from a hard error. We
    // might get here because recovery might do a flush and install a new
    // super version, which will try to schedule pending compactions. Bail
    // out here and let the higher level recovery handle compactions
    return;
S
Siying Dong 已提交
2012 2013 2014 2015 2016
  }

  if (HasExclusiveManualCompaction()) {
    // only manual compactions are allowed to run. don't schedule automatic
    // compactions
2017
    TEST_SYNC_POINT("DBImpl::MaybeScheduleFlushOrCompaction:Conflict");
S
Siying Dong 已提交
2018 2019 2020
    return;
  }

2021
  while (bg_compaction_scheduled_ < bg_job_limits.max_compactions &&
S
Siying Dong 已提交
2022 2023 2024
         unscheduled_compactions_ > 0) {
    CompactionArg* ca = new CompactionArg;
    ca->db = this;
2025
    ca->prepicked_compaction = nullptr;
S
Siying Dong 已提交
2026 2027 2028
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
    env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this,
2029
                   &DBImpl::UnscheduleCompactionCallback);
S
Siying Dong 已提交
2030 2031 2032
  }
}

2033
DBImpl::BGJobLimits DBImpl::GetBGJobLimits() const {
S
Siying Dong 已提交
2034
  mutex_.AssertHeld();
2035
  return GetBGJobLimits(mutable_db_options_.max_background_flushes,
2036 2037 2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050
                        mutable_db_options_.max_background_compactions,
                        mutable_db_options_.max_background_jobs,
                        write_controller_.NeedSpeedupCompaction());
}

DBImpl::BGJobLimits DBImpl::GetBGJobLimits(int max_background_flushes,
                                           int max_background_compactions,
                                           int max_background_jobs,
                                           bool parallelize_compactions) {
  BGJobLimits res;
  if (max_background_flushes == -1 && max_background_compactions == -1) {
    // for our first stab implementing max_background_jobs, simply allocate a
    // quarter of the threads to flushes.
    res.max_flushes = std::max(1, max_background_jobs / 4);
    res.max_compactions = std::max(1, max_background_jobs - res.max_flushes);
S
Siying Dong 已提交
2051
  } else {
2052 2053 2054 2055 2056 2057 2058 2059
    // compatibility code in case users haven't migrated to max_background_jobs,
    // which automatically computes flush/compaction limits
    res.max_flushes = std::max(1, max_background_flushes);
    res.max_compactions = std::max(1, max_background_compactions);
  }
  if (!parallelize_compactions) {
    // throttle background compactions until we deem necessary
    res.max_compactions = 1;
S
Siying Dong 已提交
2060
  }
2061
  return res;
S
Siying Dong 已提交
2062 2063 2064
}

void DBImpl::AddToCompactionQueue(ColumnFamilyData* cfd) {
2065
  assert(!cfd->queued_for_compaction());
S
Siying Dong 已提交
2066 2067
  cfd->Ref();
  compaction_queue_.push_back(cfd);
2068
  cfd->set_queued_for_compaction(true);
S
Siying Dong 已提交
2069 2070 2071 2072 2073 2074
}

ColumnFamilyData* DBImpl::PopFirstFromCompactionQueue() {
  assert(!compaction_queue_.empty());
  auto cfd = *compaction_queue_.begin();
  compaction_queue_.pop_front();
2075 2076
  assert(cfd->queued_for_compaction());
  cfd->set_queued_for_compaction(false);
S
Siying Dong 已提交
2077 2078 2079
  return cfd;
}

2080
DBImpl::FlushRequest DBImpl::PopFirstFromFlushQueue() {
S
Siying Dong 已提交
2081
  assert(!flush_queue_.empty());
2082
  FlushRequest flush_req = flush_queue_.front();
S
Siying Dong 已提交
2083
  flush_queue_.pop_front();
2084
  // TODO: need to unset flush reason?
2085
  return flush_req;
S
Siying Dong 已提交
2086 2087
}

2088 2089 2090 2091 2092 2093 2094 2095 2096 2097 2098 2099 2100 2101 2102 2103 2104 2105 2106
ColumnFamilyData* DBImpl::PickCompactionFromQueue(
    std::unique_ptr<TaskLimiterToken>* token, LogBuffer* log_buffer) {
  assert(!compaction_queue_.empty());
  assert(*token == nullptr);
  autovector<ColumnFamilyData*> throttled_candidates;
  ColumnFamilyData* cfd = nullptr;
  while (!compaction_queue_.empty()) {
    auto first_cfd = *compaction_queue_.begin();
    compaction_queue_.pop_front();
    assert(first_cfd->queued_for_compaction());
    if (!RequestCompactionToken(first_cfd, false, token, log_buffer)) {
      throttled_candidates.push_back(first_cfd);
      continue;
    }
    cfd = first_cfd;
    cfd->set_queued_for_compaction(false);
    break;
  }
  // Add throttled compaction candidates back to queue in the original order.
Y
Yanqin Jin 已提交
2107 2108
  for (auto iter = throttled_candidates.rbegin();
       iter != throttled_candidates.rend(); ++iter) {
2109 2110 2111 2112 2113
    compaction_queue_.push_front(*iter);
  }
  return cfd;
}

2114
void DBImpl::SchedulePendingFlush(const FlushRequest& flush_req,
2115
                                  FlushReason flush_reason) {
2116 2117 2118 2119 2120 2121 2122
  if (flush_req.empty()) {
    return;
  }
  for (auto& iter : flush_req) {
    ColumnFamilyData* cfd = iter.first;
    cfd->Ref();
    cfd->SetFlushReason(flush_reason);
S
Siying Dong 已提交
2123
  }
2124
  ++unscheduled_flushes_;
2125
  flush_queue_.push_back(flush_req);
S
Siying Dong 已提交
2126 2127 2128
}

void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) {
2129
  if (!cfd->queued_for_compaction() && cfd->NeedsCompaction()) {
S
Siying Dong 已提交
2130 2131 2132 2133 2134
    AddToCompactionQueue(cfd);
    ++unscheduled_compactions_;
  }
}

2135 2136
void DBImpl::SchedulePendingPurge(std::string fname, std::string dir_to_sync,
                                  FileType type, uint64_t number, int job_id) {
S
Siying Dong 已提交
2137
  mutex_.AssertHeld();
2138
  PurgeFileInfo file_info(fname, dir_to_sync, type, number, job_id);
Y
Yi Wu 已提交
2139
  purge_files_.insert({{number, std::move(file_info)}});
S
Siying Dong 已提交
2140 2141
}

2142 2143 2144 2145 2146
void DBImpl::BGWorkFlush(void* arg) {
  FlushThreadArg fta = *(reinterpret_cast<FlushThreadArg*>(arg));
  delete reinterpret_cast<FlushThreadArg*>(arg);

  IOSTATS_SET_THREAD_POOL_ID(fta.thread_pri_);
S
Siying Dong 已提交
2147
  TEST_SYNC_POINT("DBImpl::BGWorkFlush");
2148
  static_cast_with_check<DBImpl>(fta.db_)->BackgroundCallFlush(fta.thread_pri_);
S
Siying Dong 已提交
2149 2150 2151 2152 2153 2154 2155 2156
  TEST_SYNC_POINT("DBImpl::BGWorkFlush:done");
}

void DBImpl::BGWorkCompaction(void* arg) {
  CompactionArg ca = *(reinterpret_cast<CompactionArg*>(arg));
  delete reinterpret_cast<CompactionArg*>(arg);
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::LOW);
  TEST_SYNC_POINT("DBImpl::BGWorkCompaction");
2157 2158
  auto prepicked_compaction =
      static_cast<PrepickedCompaction*>(ca.prepicked_compaction);
2159
  static_cast_with_check<DBImpl>(ca.db)->BackgroundCallCompaction(
2160 2161 2162 2163 2164 2165 2166 2167 2168 2169 2170 2171 2172 2173
      prepicked_compaction, Env::Priority::LOW);
  delete prepicked_compaction;
}

void DBImpl::BGWorkBottomCompaction(void* arg) {
  CompactionArg ca = *(static_cast<CompactionArg*>(arg));
  delete static_cast<CompactionArg*>(arg);
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::BOTTOM);
  TEST_SYNC_POINT("DBImpl::BGWorkBottomCompaction");
  auto* prepicked_compaction = ca.prepicked_compaction;
  assert(prepicked_compaction && prepicked_compaction->compaction &&
         !prepicked_compaction->manual_compaction_state);
  ca.db->BackgroundCallCompaction(prepicked_compaction, Env::Priority::BOTTOM);
  delete prepicked_compaction;
S
Siying Dong 已提交
2174 2175 2176 2177 2178 2179 2180 2181 2182
}

void DBImpl::BGWorkPurge(void* db) {
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
  TEST_SYNC_POINT("DBImpl::BGWorkPurge:start");
  reinterpret_cast<DBImpl*>(db)->BackgroundCallPurge();
  TEST_SYNC_POINT("DBImpl::BGWorkPurge:end");
}

2183
void DBImpl::UnscheduleCompactionCallback(void* arg) {
S
Siying Dong 已提交
2184 2185
  CompactionArg ca = *(reinterpret_cast<CompactionArg*>(arg));
  delete reinterpret_cast<CompactionArg*>(arg);
2186 2187 2188 2189 2190
  if (ca.prepicked_compaction != nullptr) {
    if (ca.prepicked_compaction->compaction != nullptr) {
      delete ca.prepicked_compaction->compaction;
    }
    delete ca.prepicked_compaction;
S
Siying Dong 已提交
2191
  }
2192 2193 2194 2195 2196 2197
  TEST_SYNC_POINT("DBImpl::UnscheduleCompactionCallback");
}

void DBImpl::UnscheduleFlushCallback(void* arg) {
  delete reinterpret_cast<FlushThreadArg*>(arg);
  TEST_SYNC_POINT("DBImpl::UnscheduleFlushCallback");
S
Siying Dong 已提交
2198 2199 2200
}

Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
2201 2202
                               LogBuffer* log_buffer, FlushReason* reason,
                               Env::Priority thread_pri) {
S
Siying Dong 已提交
2203 2204
  mutex_.AssertHeld();

2205
  Status status;
2206 2207 2208
  *reason = FlushReason::kOthers;
  // If BG work is stopped due to an error, but a recovery is in progress,
  // that means this flush is part of the recovery. So allow it to go through
2209 2210 2211 2212
  if (!error_handler_.IsBGWorkStopped()) {
    if (shutting_down_.load(std::memory_order_acquire)) {
      status = Status::ShutdownInProgress();
    }
2213
  } else if (!error_handler_.IsRecoveryInProgress()) {
2214
    status = error_handler_.GetBGError();
S
Siying Dong 已提交
2215 2216 2217 2218 2219 2220
  }

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

2221 2222 2223
  autovector<BGFlushArg> bg_flush_args;
  std::vector<SuperVersionContext>& superversion_contexts =
      job_context->superversion_contexts;
2224
  autovector<ColumnFamilyData*> column_families_not_to_flush;
S
Siying Dong 已提交
2225 2226
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
2227 2228 2229 2230 2231 2232 2233 2234
    const FlushRequest& flush_req = PopFirstFromFlushQueue();
    superversion_contexts.clear();
    superversion_contexts.reserve(flush_req.size());

    for (const auto& iter : flush_req) {
      ColumnFamilyData* cfd = iter.first;
      if (cfd->IsDropped() || !cfd->imm()->IsFlushPending()) {
        // can't flush this CF, try next one
2235
        column_families_not_to_flush.push_back(cfd);
2236
        continue;
S
Siying Dong 已提交
2237
      }
2238 2239 2240 2241 2242 2243
      superversion_contexts.emplace_back(SuperVersionContext(true));
      bg_flush_args.emplace_back(cfd, iter.second,
                                 &(superversion_contexts.back()));
    }
    if (!bg_flush_args.empty()) {
      break;
S
Siying Dong 已提交
2244 2245 2246
    }
  }

2247
  if (!bg_flush_args.empty()) {
2248
    auto bg_job_limits = GetBGJobLimits();
2249 2250 2251 2252 2253 2254 2255 2256 2257 2258 2259 2260 2261
    for (const auto& arg : bg_flush_args) {
      ColumnFamilyData* cfd = arg.cfd_;
      ROCKS_LOG_BUFFER(
          log_buffer,
          "Calling FlushMemTableToOutputFile with column "
          "family [%s], flush slots available %d, compaction slots available "
          "%d, "
          "flush slots scheduled %d, compaction slots scheduled %d",
          cfd->GetName().c_str(), bg_job_limits.max_flushes,
          bg_job_limits.max_compactions, bg_flush_scheduled_,
          bg_compaction_scheduled_);
    }
    status = FlushMemTablesToOutputFiles(bg_flush_args, made_progress,
2262
                                         job_context, log_buffer, thread_pri);
2263
    TEST_SYNC_POINT("DBImpl::BackgroundFlush:BeforeFlush");
2264 2265 2266
    // All the CFDs in the FlushReq must have the same flush reason, so just
    // grab the first one
    *reason = bg_flush_args[0].cfd_->GetFlushReason();
2267 2268
    for (auto& arg : bg_flush_args) {
      ColumnFamilyData* cfd = arg.cfd_;
2269
      if (cfd->UnrefAndTryDelete()) {
2270 2271
        arg.cfd_ = nullptr;
      }
S
Siying Dong 已提交
2272 2273
    }
  }
2274
  for (auto cfd : column_families_not_to_flush) {
2275
    cfd->UnrefAndTryDelete();
2276
  }
S
Siying Dong 已提交
2277 2278 2279
  return status;
}

2280
void DBImpl::BackgroundCallFlush(Env::Priority thread_pri) {
S
Siying Dong 已提交
2281 2282 2283 2284 2285 2286 2287 2288 2289
  bool made_progress = false;
  JobContext job_context(next_job_id_.fetch_add(1), true);

  TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:start");

  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
                       immutable_db_options_.info_log.get());
  {
    InstrumentedMutexLock l(&mutex_);
A
Andrew Kryczka 已提交
2290
    assert(bg_flush_scheduled_);
S
Siying Dong 已提交
2291 2292
    num_running_flushes_++;

2293 2294 2295
    std::unique_ptr<std::list<uint64_t>::iterator>
        pending_outputs_inserted_elem(new std::list<uint64_t>::iterator(
            CaptureCurrentFileNumberInPendingOutputs()));
2296
    FlushReason reason;
S
Siying Dong 已提交
2297

2298 2299
    Status s = BackgroundFlush(&made_progress, &job_context, &log_buffer,
                               &reason, thread_pri);
2300
    if (!s.ok() && !s.IsShutdownInProgress() && !s.IsColumnFamilyDropped() &&
2301
        reason != FlushReason::kErrorRecovery) {
S
Siying Dong 已提交
2302 2303 2304 2305 2306
      // Wait a little bit before retrying background flush in
      // case this is an environmental problem and we do not want to
      // chew up resources for failed flushes for the duration of
      // the problem.
      uint64_t error_cnt =
2307
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
S
Siying Dong 已提交
2308 2309 2310 2311 2312 2313 2314 2315 2316 2317 2318 2319
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
      ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                      "Waiting after background flush error: %s"
                      "Accumulated background error counts: %" PRIu64,
                      s.ToString().c_str(), error_cnt);
      log_buffer.FlushBufferToLog();
      LogFlush(immutable_db_options_.info_log);
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
    }

Y
Yanqin Jin 已提交
2320
    TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:FlushFinish:0");
S
Siying Dong 已提交
2321 2322 2323 2324
    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

    // If flush failed, we want to delete all temporary files that we might have
    // created. Thus, we force full scan in FindObsoleteFiles()
2325 2326
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress() &&
                                        !s.IsColumnFamilyDropped());
S
Siying Dong 已提交
2327
    // delete unnecessary files if any, this is done outside the mutex
2328 2329
    if (job_context.HaveSomethingToClean() ||
        job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
S
Siying Dong 已提交
2330
      mutex_.Unlock();
2331
      TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:FilesFound");
S
Siying Dong 已提交
2332 2333 2334 2335 2336 2337 2338 2339 2340 2341 2342 2343
      // Have to flush the info logs before bg_flush_scheduled_--
      // because if bg_flush_scheduled_ becomes 0 and the lock is
      // released, the deconstructor of DB can kick in and destroy all the
      // states of DB so info_log might not be available after that point.
      // It also applies to access other states that DB owns.
      log_buffer.FlushBufferToLog();
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
      }
      job_context.Clean();
      mutex_.Lock();
    }
Y
Yi Wu 已提交
2344
    TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:ContextCleanedUp");
S
Siying Dong 已提交
2345 2346 2347 2348 2349 2350

    assert(num_running_flushes_ > 0);
    num_running_flushes_--;
    bg_flush_scheduled_--;
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2351
    atomic_flush_install_cv_.SignalAll();
S
Siying Dong 已提交
2352 2353 2354 2355 2356 2357 2358 2359
    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.
  }
}

2360 2361
void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction,
                                      Env::Priority bg_thread_pri) {
S
Siying Dong 已提交
2362 2363 2364 2365 2366 2367 2368 2369 2370 2371 2372 2373 2374 2375
  bool made_progress = false;
  JobContext job_context(next_job_id_.fetch_add(1), true);
  TEST_SYNC_POINT("BackgroundCallCompaction:0");
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
                       immutable_db_options_.info_log.get());
  {
    InstrumentedMutexLock l(&mutex_);

    // This call will unlock/lock the mutex to wait for current running
    // IngestExternalFile() calls to finish.
    WaitForIngestFile();

    num_running_compactions_++;

2376 2377 2378
    std::unique_ptr<std::list<uint64_t>::iterator>
        pending_outputs_inserted_elem(new std::list<uint64_t>::iterator(
            CaptureCurrentFileNumberInPendingOutputs()));
S
Siying Dong 已提交
2379

2380 2381 2382 2383
    assert((bg_thread_pri == Env::Priority::BOTTOM &&
            bg_bottom_compaction_scheduled_) ||
           (bg_thread_pri == Env::Priority::LOW && bg_compaction_scheduled_));
    Status s = BackgroundCompaction(&made_progress, &job_context, &log_buffer,
2384
                                    prepicked_compaction, bg_thread_pri);
S
Siying Dong 已提交
2385
    TEST_SYNC_POINT("BackgroundCallCompaction:1");
2386 2387 2388
    if (s.IsBusy()) {
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
2389
      env_->SleepForMicroseconds(10000);  // prevent hot loop
2390
      mutex_.Lock();
2391
    } else if (!s.ok() && !s.IsShutdownInProgress() &&
2392
               !s.IsManualCompactionPaused() && !s.IsColumnFamilyDropped()) {
S
Siying Dong 已提交
2393 2394 2395 2396 2397 2398 2399 2400 2401 2402 2403 2404 2405 2406 2407 2408
      // Wait a little bit before retrying background compaction in
      // case this is an environmental problem and we do not want to
      // chew up resources for failed compactions for the duration of
      // the problem.
      uint64_t error_cnt =
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
      log_buffer.FlushBufferToLog();
      ROCKS_LOG_ERROR(immutable_db_options_.info_log,
                      "Waiting after background compaction error: %s, "
                      "Accumulated background error counts: %" PRIu64,
                      s.ToString().c_str(), error_cnt);
      LogFlush(immutable_db_options_.info_log);
      env_->SleepForMicroseconds(1000000);
      mutex_.Lock();
2409
    } else if (s.IsManualCompactionPaused()) {
2410
      ManualCompactionState* m = prepicked_compaction->manual_compaction_state;
2411 2412
      assert(m);
      ROCKS_LOG_BUFFER(&log_buffer, "[%s] [JOB %d] Manual compaction paused",
2413
                       m->cfd->GetName().c_str(), job_context.job_id);
S
Siying Dong 已提交
2414 2415 2416 2417 2418 2419 2420
    }

    ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

    // If compaction failed, we want to delete all temporary files that we might
    // have created (they might not be all recorded in job_context in case of a
    // failure). Thus, we force full scan in FindObsoleteFiles()
2421
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress() &&
2422 2423
                                        !s.IsManualCompactionPaused() &&
                                        !s.IsColumnFamilyDropped());
2424
    TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:FoundObsoleteFiles");
S
Siying Dong 已提交
2425 2426

    // delete unnecessary files if any, this is done outside the mutex
2427 2428
    if (job_context.HaveSomethingToClean() ||
        job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
S
Siying Dong 已提交
2429 2430 2431 2432 2433 2434 2435 2436 2437
      mutex_.Unlock();
      // Have to flush the info logs before bg_compaction_scheduled_--
      // because if bg_flush_scheduled_ becomes 0 and the lock is
      // released, the deconstructor of DB can kick in and destroy all the
      // states of DB so info_log might not be available after that point.
      // It also applies to access other states that DB owns.
      log_buffer.FlushBufferToLog();
      if (job_context.HaveSomethingToDelete()) {
        PurgeObsoleteFiles(job_context);
2438
        TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:PurgedObsoleteFiles");
S
Siying Dong 已提交
2439 2440 2441 2442 2443 2444 2445
      }
      job_context.Clean();
      mutex_.Lock();
    }

    assert(num_running_compactions_ > 0);
    num_running_compactions_--;
2446 2447 2448 2449 2450 2451
    if (bg_thread_pri == Env::Priority::LOW) {
      bg_compaction_scheduled_--;
    } else {
      assert(bg_thread_pri == Env::Priority::BOTTOM);
      bg_bottom_compaction_scheduled_--;
    }
S
Siying Dong 已提交
2452 2453 2454 2455 2456

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

    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2457 2458 2459
    if (made_progress ||
        (bg_compaction_scheduled_ == 0 &&
         bg_bottom_compaction_scheduled_ == 0) ||
2460
        HasPendingManualCompaction() || unscheduled_compactions_ == 0) {
S
Siying Dong 已提交
2461 2462
      // signal if
      // * made_progress -- need to wakeup DelayWrite
2463
      // * bg_{bottom,}_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl
S
Siying Dong 已提交
2464 2465 2466 2467 2468 2469 2470 2471 2472 2473 2474 2475 2476 2477
      // * HasPendingManualCompaction -- need to wakeup RunManualCompaction
      // If none of this is true, there is no need to signal since nobody is
      // waiting for it
      bg_cv_.SignalAll();
    }
    // 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.
  }
}

Status DBImpl::BackgroundCompaction(bool* made_progress,
                                    JobContext* job_context,
2478
                                    LogBuffer* log_buffer,
2479 2480
                                    PrepickedCompaction* prepicked_compaction,
                                    Env::Priority thread_pri) {
2481 2482 2483 2484
  ManualCompactionState* manual_compaction =
      prepicked_compaction == nullptr
          ? nullptr
          : prepicked_compaction->manual_compaction_state;
S
Siying Dong 已提交
2485 2486 2487 2488 2489
  *made_progress = false;
  mutex_.AssertHeld();
  TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Start");

  bool is_manual = (manual_compaction != nullptr);
2490
  std::unique_ptr<Compaction> c;
2491 2492 2493 2494 2495
  if (prepicked_compaction != nullptr &&
      prepicked_compaction->compaction != nullptr) {
    c.reset(prepicked_compaction->compaction);
  }
  bool is_prepicked = is_manual || c;
S
Siying Dong 已提交
2496 2497 2498 2499 2500 2501

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

  CompactionJobStats compaction_job_stats;
2502 2503 2504 2505
  Status status;
  if (!error_handler_.IsBGWorkStopped()) {
    if (shutting_down_.load(std::memory_order_acquire)) {
      status = Status::ShutdownInProgress();
2506
    } else if (is_manual &&
2507
               manual_compaction_paused_.load(std::memory_order_acquire)) {
2508
      status = Status::Incomplete(Status::SubCode::kManualCompactionPaused);
2509 2510 2511
    }
  } else {
    status = error_handler_.GetBGError();
2512 2513 2514 2515 2516
    // If we get here, it means a hard error happened after this compaction
    // was scheduled by MaybeScheduleFlushOrCompaction(), but before it got
    // a chance to execute. Since we didn't pop a cfd from the compaction
    // queue, increment unscheduled_compactions_
    unscheduled_compactions_++;
S
Siying Dong 已提交
2517 2518 2519 2520 2521 2522 2523 2524 2525
  }

  if (!status.ok()) {
    if (is_manual) {
      manual_compaction->status = status;
      manual_compaction->done = true;
      manual_compaction->in_progress = false;
      manual_compaction = nullptr;
    }
2526 2527 2528 2529
    if (c) {
      c->ReleaseCompactionFiles(status);
      c.reset();
    }
S
Siying Dong 已提交
2530 2531 2532 2533 2534 2535 2536 2537
    return status;
  }

  if (is_manual) {
    // another thread cannot pick up the same work
    manual_compaction->in_progress = true;
  }

2538 2539
  std::unique_ptr<TaskLimiterToken> task_token;

S
Siying Dong 已提交
2540 2541
  // InternalKey manual_end_storage;
  // InternalKey* manual_end = &manual_end_storage;
2542
  bool sfm_reserved_compact_space = false;
S
Siying Dong 已提交
2543
  if (is_manual) {
2544
    ManualCompactionState* m = manual_compaction;
S
Siying Dong 已提交
2545 2546 2547 2548 2549 2550 2551 2552
    assert(m->in_progress);
    if (!c) {
      m->done = true;
      m->manual_end = nullptr;
      ROCKS_LOG_BUFFER(log_buffer,
                       "[%s] Manual compaction from level-%d from %s .. "
                       "%s; nothing to do\n",
                       m->cfd->GetName().c_str(), m->input_level,
2553 2554
                       (m->begin ? m->begin->DebugString(true).c_str() : "(begin)"),
                       (m->end ? m->end->DebugString(true).c_str() : "(end)"));
S
Siying Dong 已提交
2555
    } else {
2556 2557
      // First check if we have enough room to do the compaction
      bool enough_room = EnoughRoomForCompaction(
2558
          m->cfd, *(c->inputs()), &sfm_reserved_compact_space, log_buffer);
2559 2560 2561 2562 2563 2564 2565 2566 2567 2568 2569 2570 2571 2572

      if (!enough_room) {
        // Then don't do the compaction
        c->ReleaseCompactionFiles(status);
        c.reset();
        // m's vars will get set properly at the end of this function,
        // as long as status == CompactionTooLarge
        status = Status::CompactionTooLarge();
      } else {
        ROCKS_LOG_BUFFER(
            log_buffer,
            "[%s] Manual compaction from level-%d to level-%d from %s .. "
            "%s; will stop at %s\n",
            m->cfd->GetName().c_str(), m->input_level, c->output_level(),
2573 2574
            (m->begin ? m->begin->DebugString(true).c_str() : "(begin)"),
            (m->end ? m->end->DebugString(true).c_str() : "(end)"),
2575 2576
            ((m->done || m->manual_end == nullptr)
                 ? "(end)"
2577
                 : m->manual_end->DebugString(true).c_str()));
2578
      }
S
Siying Dong 已提交
2579
    }
2580
  } else if (!is_prepicked && !compaction_queue_.empty()) {
2581
    if (HasExclusiveManualCompaction()) {
2582 2583 2584
      // Can't compact right now, but try again later
      TEST_SYNC_POINT("DBImpl::BackgroundCompaction()::Conflict");

Y
yizhu.sun 已提交
2585
      // Stay in the compaction queue.
2586 2587 2588 2589 2590
      unscheduled_compactions_++;

      return Status::OK();
    }

2591 2592 2593 2594 2595 2596 2597 2598
    auto cfd = PickCompactionFromQueue(&task_token, log_buffer);
    if (cfd == nullptr) {
      // Can't find any executable task from the compaction queue.
      // All tasks have been throttled by compaction thread limiter.
      ++unscheduled_compactions_;
      return Status::Busy();
    }

S
Siying Dong 已提交
2599 2600 2601 2602 2603
    // We unreference here because the following code will take a Ref() on
    // this cfd if it is going to use it (Compaction class holds a
    // reference).
    // This will all happen under a mutex so we don't have to be afraid of
    // somebody else deleting it.
2604
    if (cfd->UnrefAndTryDelete()) {
S
Siying Dong 已提交
2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622
      // This was the last reference of the column family, so no need to
      // compact.
      return Status::OK();
    }

    // Pick up latest mutable CF Options and use it throughout the
    // compaction job
    // Compaction makes a copy of the latest MutableCFOptions. It should be used
    // throughout the compaction procedure to make sure consistency. It will
    // eventually be installed into SuperVersion
    auto* mutable_cf_options = cfd->GetLatestMutableCFOptions();
    if (!mutable_cf_options->disable_auto_compactions && !cfd->IsDropped()) {
      // NOTE: try to avoid unnecessary copy of MutableCFOptions if
      // compaction is not necessary. Need to make sure mutex is held
      // until we make a copy in the following code
      TEST_SYNC_POINT("DBImpl::BackgroundCompaction():BeforePickCompaction");
      c.reset(cfd->PickCompaction(*mutable_cf_options, log_buffer));
      TEST_SYNC_POINT("DBImpl::BackgroundCompaction():AfterPickCompaction");
2623

S
Siying Dong 已提交
2624
      if (c != nullptr) {
2625
        bool enough_room = EnoughRoomForCompaction(
2626
            cfd, *(c->inputs()), &sfm_reserved_compact_space, log_buffer);
2627

2628 2629 2630 2631 2632 2633 2634 2635
        if (!enough_room) {
          // Then don't do the compaction
          c->ReleaseCompactionFiles(status);
          c->column_family_data()
              ->current()
              ->storage_info()
              ->ComputeCompactionScore(*(c->immutable_cf_options()),
                                       *(c->mutable_cf_options()));
S
Siying Dong 已提交
2636 2637
          AddToCompactionQueue(cfd);
          ++unscheduled_compactions_;
2638 2639 2640 2641 2642 2643 2644

          c.reset();
          // Don't need to sleep here, because BackgroundCallCompaction
          // will sleep if !s.ok()
          status = Status::CompactionTooLarge();
        } else {
          // update statistics
S
Siying Dong 已提交
2645 2646
          RecordInHistogram(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
                            c->inputs(0)->size());
2647 2648 2649 2650 2651 2652 2653 2654 2655 2656 2657 2658 2659 2660 2661 2662 2663 2664
          // There are three things that can change compaction score:
          // 1) When flush or compaction finish. This case is covered by
          // InstallSuperVersionAndScheduleWork
          // 2) When MutableCFOptions changes. This case is also covered by
          // InstallSuperVersionAndScheduleWork, because this is when the new
          // options take effect.
          // 3) When we Pick a new compaction, we "remove" those files being
          // compacted from the calculation, which then influences compaction
          // score. Here we check if we need the new compaction even without the
          // files that are currently being compacted. If we need another
          // compaction, we might be able to execute it in parallel, so we add
          // it to the queue and schedule a new thread.
          if (cfd->NeedsCompaction()) {
            // Yes, we need more compactions!
            AddToCompactionQueue(cfd);
            ++unscheduled_compactions_;
            MaybeScheduleFlushOrCompaction();
          }
S
Siying Dong 已提交
2665 2666 2667 2668 2669
        }
      }
    }
  }

2670
  IOStatus io_s;
S
Siying Dong 已提交
2671 2672 2673 2674 2675 2676
  if (!c) {
    // Nothing to do
    ROCKS_LOG_BUFFER(log_buffer, "Compaction nothing to do");
  } else if (c->deletion_compaction()) {
    // TODO(icanadi) Do we want to honor snapshots here? i.e. not delete old
    // file if there is alive snapshot pointing to it
2677 2678
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2679 2680 2681 2682 2683 2684 2685
    assert(c->num_input_files(1) == 0);
    assert(c->level() == 0);
    assert(c->column_family_data()->ioptions()->compaction_style ==
           kCompactionStyleFIFO);

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

P
Peter Pei 已提交
2686 2687 2688
    NotifyOnCompactionBegin(c->column_family_data(), c.get(), status,
                            compaction_job_stats, job_context->job_id);

S
Siying Dong 已提交
2689 2690 2691
    for (const auto& f : *c->inputs(0)) {
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
    }
2692
    versions_->SetIOStatusOK();
S
Siying Dong 已提交
2693 2694 2695
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2696
    io_s = versions_->io_status();
Y
Yanqin Jin 已提交
2697 2698 2699
    InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                       &job_context->superversion_contexts[0],
                                       *c->mutable_cf_options());
S
Siying Dong 已提交
2700 2701 2702 2703
    ROCKS_LOG_BUFFER(log_buffer, "[%s] Deleted %d files\n",
                     c->column_family_data()->GetName().c_str(),
                     c->num_input_files(0));
    *made_progress = true;
2704 2705
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2706 2707
  } else if (!trivial_move_disallowed && c->IsTrivialMove()) {
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2708 2709
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2710 2711 2712 2713 2714 2715 2716 2717 2718
    // Instrument for event update
    // TODO(yhchiang): add op details for showing trivial-move.
    ThreadStatusUtil::SetColumnFamily(
        c->column_family_data(), c->column_family_data()->ioptions()->env,
        immutable_db_options_.enable_thread_tracking);
    ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);

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

P
Peter Pei 已提交
2719 2720 2721
    NotifyOnCompactionBegin(c->column_family_data(), c.get(), status,
                            compaction_job_stats, job_context->job_id);

S
Siying Dong 已提交
2722 2723 2724 2725 2726 2727 2728 2729 2730 2731 2732 2733
    // Move files to next level
    int32_t moved_files = 0;
    int64_t moved_bytes = 0;
    for (unsigned int l = 0; l < c->num_input_levels(); l++) {
      if (c->level(l) == c->output_level()) {
        continue;
      }
      for (size_t i = 0; i < c->num_input_files(l); i++) {
        FileMetaData* f = c->input(l, i);
        c->edit()->DeleteFile(c->level(l), f->fd.GetNumber());
        c->edit()->AddFile(c->output_level(), f->fd.GetNumber(),
                           f->fd.GetPathId(), f->fd.GetFileSize(), f->smallest,
2734
                           f->largest, f->fd.smallest_seqno,
2735
                           f->fd.largest_seqno, f->marked_for_compaction,
2736
                           f->oldest_blob_file_number, f->oldest_ancester_time,
2737 2738
                           f->file_creation_time, f->file_checksum,
                           f->file_checksum_func_name);
S
Siying Dong 已提交
2739

2740 2741 2742 2743 2744
        ROCKS_LOG_BUFFER(
            log_buffer,
            "[%s] Moving #%" PRIu64 " to level-%d %" PRIu64 " bytes\n",
            c->column_family_data()->GetName().c_str(), f->fd.GetNumber(),
            c->output_level(), f->fd.GetFileSize());
S
Siying Dong 已提交
2745 2746 2747 2748 2749
        ++moved_files;
        moved_bytes += f->fd.GetFileSize();
      }
    }

2750
    versions_->SetIOStatusOK();
S
Siying Dong 已提交
2751 2752 2753
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
2754
    io_s = versions_->io_status();
S
Siying Dong 已提交
2755
    // Use latest MutableCFOptions
Y
Yanqin Jin 已提交
2756 2757 2758
    InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                       &job_context->superversion_contexts[0],
                                       *c->mutable_cf_options());
S
Siying Dong 已提交
2759 2760 2761 2762 2763 2764 2765 2766 2767 2768 2769 2770 2771 2772 2773 2774 2775 2776 2777 2778 2779

    VersionStorageInfo::LevelSummaryStorage tmp;
    c->column_family_data()->internal_stats()->IncBytesMoved(c->output_level(),
                                                             moved_bytes);
    {
      event_logger_.LogToBuffer(log_buffer)
          << "job" << job_context->job_id << "event"
          << "trivial_move"
          << "destination_level" << c->output_level() << "files" << moved_files
          << "total_files_size" << moved_bytes;
    }
    ROCKS_LOG_BUFFER(
        log_buffer,
        "[%s] Moved #%d files to level-%d %" PRIu64 " bytes %s: %s\n",
        c->column_family_data()->GetName().c_str(), moved_files,
        c->output_level(), moved_bytes, status.ToString().c_str(),
        c->column_family_data()->current()->storage_info()->LevelSummary(&tmp));
    *made_progress = true;

    // Clear Instrument
    ThreadStatusUtil::ResetThreadStatus();
2780 2781
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction",
                             c->column_family_data());
2782
  } else if (!is_prepicked && c->output_level() > 0 &&
2783 2784 2785 2786 2787 2788 2789
             c->output_level() ==
                 c->column_family_data()
                     ->current()
                     ->storage_info()
                     ->MaxOutputLevel(
                         immutable_db_options_.allow_ingest_behind) &&
             env_->GetBackgroundThreads(Env::Priority::BOTTOM) > 0) {
2790 2791 2792
    // Forward compactions involving last level to the bottom pool if it exists,
    // such that compactions unlikely to contribute to write stalls can be
    // delayed or deprioritized.
2793 2794 2795 2796 2797 2798
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:ForwardToBottomPriPool");
    CompactionArg* ca = new CompactionArg;
    ca->db = this;
    ca->prepicked_compaction = new PrepickedCompaction;
    ca->prepicked_compaction->compaction = c.release();
    ca->prepicked_compaction->manual_compaction_state = nullptr;
2799 2800
    // Transfer requested token, so it doesn't need to do it again.
    ca->prepicked_compaction->task_token = std::move(task_token);
2801 2802
    ++bg_bottom_compaction_scheduled_;
    env_->Schedule(&DBImpl::BGWorkBottomCompaction, ca, Env::Priority::BOTTOM,
2803
                   this, &DBImpl::UnscheduleCompactionCallback);
S
Siying Dong 已提交
2804
  } else {
2805 2806
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction",
                             c->column_family_data());
2807
    int output_level __attribute__((__unused__));
2808
    output_level = c->output_level();
S
Siying Dong 已提交
2809 2810
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial",
                             &output_level);
2811
    std::vector<SequenceNumber> snapshot_seqs;
S
Siying Dong 已提交
2812
    SequenceNumber earliest_write_conflict_snapshot;
2813 2814 2815
    SnapshotChecker* snapshot_checker;
    GetSnapshotContext(job_context, &snapshot_seqs,
                       &earliest_write_conflict_snapshot, &snapshot_checker);
S
Siying Dong 已提交
2816 2817
    assert(is_snapshot_supported_ || snapshots_.empty());
    CompactionJob compaction_job(
2818
        job_context->job_id, c.get(), immutable_db_options_,
2819
        file_options_for_compaction_, versions_.get(), &shutting_down_,
2820
        preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
2821
        GetDataDir(c->column_family_data(), c->output_path_id()), stats_,
2822 2823 2824
        &mutex_, &error_handler_, snapshot_seqs,
        earliest_write_conflict_snapshot, snapshot_checker, table_cache_,
        &event_logger_, c->mutable_cf_options()->paranoid_file_checks,
S
Siying Dong 已提交
2825
        c->mutable_cf_options()->report_bg_io_stats, dbname_,
2826
        &compaction_job_stats, thread_pri,
2827 2828
        is_manual ? &manual_compaction_paused_ : nullptr, db_id_,
        db_session_id_);
S
Siying Dong 已提交
2829 2830
    compaction_job.Prepare();

P
Peter Pei 已提交
2831 2832 2833
    NotifyOnCompactionBegin(c->column_family_data(), c.get(), status,
                            compaction_job_stats, job_context->job_id);

S
Siying Dong 已提交
2834
    mutex_.Unlock();
2835 2836
    TEST_SYNC_POINT_CALLBACK(
        "DBImpl::BackgroundCompaction:NonTrivial:BeforeRun", nullptr);
S
Siying Dong 已提交
2837 2838 2839 2840 2841
    compaction_job.Run();
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun");
    mutex_.Lock();

    status = compaction_job.Install(*c->mutable_cf_options());
2842
    io_s = compaction_job.io_status();
S
Siying Dong 已提交
2843
    if (status.ok()) {
Y
Yanqin Jin 已提交
2844 2845 2846
      InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                         &job_context->superversion_contexts[0],
                                         *c->mutable_cf_options());
S
Siying Dong 已提交
2847 2848
    }
    *made_progress = true;
2849 2850
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2851
  }
2852 2853 2854 2855 2856

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

S
Siying Dong 已提交
2857 2858 2859
  if (c != nullptr) {
    c->ReleaseCompactionFiles(status);
    *made_progress = true;
2860 2861 2862 2863 2864

#ifndef ROCKSDB_LITE
    // Need to make sure SstFileManager does its bookkeeping
    auto sfm = static_cast<SstFileManagerImpl*>(
        immutable_db_options_.sst_file_manager.get());
2865
    if (sfm && sfm_reserved_compact_space) {
2866 2867 2868 2869
      sfm->OnCompactionCompletion(c.get());
    }
#endif  // ROCKSDB_LITE

2870 2871
    NotifyOnCompactionCompleted(c->column_family_data(), c.get(), status,
                                compaction_job_stats, job_context->job_id);
S
Siying Dong 已提交
2872 2873
  }

2874 2875
  if (status.ok() || status.IsCompactionTooLarge() ||
      status.IsManualCompactionPaused()) {
S
Siying Dong 已提交
2876
    // Done
2877
  } else if (status.IsColumnFamilyDropped() || status.IsShutdownInProgress()) {
S
Siying Dong 已提交
2878 2879 2880 2881
    // Ignore compaction errors found during shutting down
  } else {
    ROCKS_LOG_WARN(immutable_db_options_.info_log, "Compaction error: %s",
                   status.ToString().c_str());
2882 2883 2884 2885 2886
    if (!io_s.ok()) {
      error_handler_.SetBGError(io_s, BackgroundErrorReason::kCompaction);
    } else {
      error_handler_.SetBGError(status, BackgroundErrorReason::kCompaction);
    }
2887 2888 2889 2890 2891 2892 2893 2894 2895 2896 2897 2898 2899 2900 2901 2902 2903
    if (c != nullptr && !is_manual && !error_handler_.IsBGWorkStopped()) {
      // Put this cfd back in the compaction queue so we can retry after some
      // time
      auto cfd = c->column_family_data();
      assert(cfd != nullptr);
      // Since this compaction failed, we need to recompute the score so it
      // takes the original input files into account
      c->column_family_data()
          ->current()
          ->storage_info()
          ->ComputeCompactionScore(*(c->immutable_cf_options()),
                                   *(c->mutable_cf_options()));
      if (!cfd->queued_for_compaction()) {
        AddToCompactionQueue(cfd);
        ++unscheduled_compactions_;
      }
    }
S
Siying Dong 已提交
2904
  }
2905 2906
  // this will unref its input_version and column_family_data
  c.reset();
S
Siying Dong 已提交
2907 2908

  if (is_manual) {
2909
    ManualCompactionState* m = manual_compaction;
S
Siying Dong 已提交
2910 2911 2912 2913 2914 2915 2916 2917 2918 2919 2920 2921 2922 2923 2924 2925 2926 2927 2928 2929 2930 2931 2932 2933 2934 2935 2936 2937 2938 2939 2940 2941
    if (!status.ok()) {
      m->status = status;
      m->done = true;
    }
    // For universal compaction:
    //   Because universal compaction always happens at level 0, so one
    //   compaction will pick up all overlapped files. No files will be
    //   filtered out due to size limit and left for a successive compaction.
    //   So we can safely conclude the current compaction.
    //
    //   Also note that, if we don't stop here, then the current compaction
    //   writes a new file back to level 0, which will be used in successive
    //   compaction. Hence the manual compaction will never finish.
    //
    // Stop the compaction if manual_end points to nullptr -- this means
    // that we compacted the whole range. manual_end should always point
    // to nullptr in case of universal compaction
    if (m->manual_end == nullptr) {
      m->done = true;
    }
    if (!m->done) {
      // We only compacted part of the requested range.  Update *m
      // to the range that is left to be compacted.
      // Universal and FIFO compactions should always compact the whole range
      assert(m->cfd->ioptions()->compaction_style !=
                 kCompactionStyleUniversal ||
             m->cfd->ioptions()->num_levels > 1);
      assert(m->cfd->ioptions()->compaction_style != kCompactionStyleFIFO);
      m->tmp_storage = *m->manual_end;
      m->begin = &m->tmp_storage;
      m->incomplete = true;
    }
2942
    m->in_progress = false;  // not being processed anymore
S
Siying Dong 已提交
2943 2944 2945 2946 2947 2948 2949 2950 2951
  }
  TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Finish");
  return status;
}

bool DBImpl::HasPendingManualCompaction() {
  return (!manual_compaction_dequeue_.empty());
}

2952
void DBImpl::AddManualCompaction(DBImpl::ManualCompactionState* m) {
S
Siying Dong 已提交
2953 2954 2955
  manual_compaction_dequeue_.push_back(m);
}

2956
void DBImpl::RemoveManualCompaction(DBImpl::ManualCompactionState* m) {
S
Siying Dong 已提交
2957
  // Remove from queue
2958
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
2959 2960 2961 2962 2963 2964
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if (m == (*it)) {
      it = manual_compaction_dequeue_.erase(it);
      return;
    }
2965
    ++it;
S
Siying Dong 已提交
2966 2967 2968 2969 2970
  }
  assert(false);
  return;
}

2971
bool DBImpl::ShouldntRunManualCompaction(ManualCompactionState* m) {
S
Siying Dong 已提交
2972 2973 2974 2975 2976 2977
  if (num_running_ingest_file_ > 0) {
    // We need to wait for other IngestExternalFile() calls to finish
    // before running a manual compaction.
    return true;
  }
  if (m->exclusive) {
2978 2979
    return (bg_bottom_compaction_scheduled_ > 0 ||
            bg_compaction_scheduled_ > 0);
S
Siying Dong 已提交
2980
  }
2981
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
2982 2983 2984 2985
      manual_compaction_dequeue_.begin();
  bool seen = false;
  while (it != manual_compaction_dequeue_.end()) {
    if (m == (*it)) {
2986
      ++it;
S
Siying Dong 已提交
2987 2988 2989 2990 2991 2992 2993 2994
      seen = true;
      continue;
    } else if (MCOverlap(m, (*it)) && (!seen && !(*it)->in_progress)) {
      // Consider the other manual compaction *it, conflicts if:
      // overlaps with m
      // and (*it) is ahead in the queue and is not yet in progress
      return true;
    }
2995
    ++it;
S
Siying Dong 已提交
2996 2997 2998 2999 3000 3001
  }
  return false;
}

bool DBImpl::HaveManualCompaction(ColumnFamilyData* cfd) {
  // Remove from priority queue
3002
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
3003 3004 3005 3006 3007 3008 3009
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if ((*it)->exclusive) {
      return true;
    }
    if ((cfd == (*it)->cfd) && (!((*it)->in_progress || (*it)->done))) {
      // Allow automatic compaction if manual compaction is
H
hyunwoo 已提交
3010
      // in progress
S
Siying Dong 已提交
3011 3012
      return true;
    }
3013
    ++it;
S
Siying Dong 已提交
3014 3015 3016 3017 3018 3019
  }
  return false;
}

bool DBImpl::HasExclusiveManualCompaction() {
  // Remove from priority queue
3020
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
3021 3022 3023 3024 3025
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if ((*it)->exclusive) {
      return true;
    }
3026
    ++it;
S
Siying Dong 已提交
3027 3028 3029 3030
  }
  return false;
}

3031
bool DBImpl::MCOverlap(ManualCompactionState* m, ManualCompactionState* m1) {
S
Siying Dong 已提交
3032 3033 3034 3035 3036 3037 3038 3039 3040
  if ((m->exclusive) || (m1->exclusive)) {
    return true;
  }
  if (m->cfd != m1->cfd) {
    return false;
  }
  return true;
}

3041 3042 3043 3044 3045 3046 3047 3048 3049 3050 3051 3052 3053 3054 3055 3056 3057 3058 3059
#ifndef ROCKSDB_LITE
void DBImpl::BuildCompactionJobInfo(
    const ColumnFamilyData* cfd, Compaction* c, const Status& st,
    const CompactionJobStats& compaction_job_stats, const int job_id,
    const Version* current, CompactionJobInfo* compaction_job_info) const {
  assert(compaction_job_info != nullptr);
  compaction_job_info->cf_id = cfd->GetID();
  compaction_job_info->cf_name = cfd->GetName();
  compaction_job_info->status = st;
  compaction_job_info->thread_id = env_->GetThreadID();
  compaction_job_info->job_id = job_id;
  compaction_job_info->base_input_level = c->start_level();
  compaction_job_info->output_level = c->output_level();
  compaction_job_info->stats = compaction_job_stats;
  compaction_job_info->table_properties = c->GetOutputTableProperties();
  compaction_job_info->compaction_reason = c->compaction_reason();
  compaction_job_info->compression = c->output_compression();
  for (size_t i = 0; i < c->num_input_levels(); ++i) {
    for (const auto fmd : *c->inputs(i)) {
3060 3061 3062 3063
      const FileDescriptor& desc = fmd->fd;
      const uint64_t file_number = desc.GetNumber();
      auto fn = TableFileName(c->immutable_cf_options()->cf_paths, file_number,
                              desc.GetPathId());
3064
      compaction_job_info->input_files.push_back(fn);
3065 3066
      compaction_job_info->input_file_infos.push_back(CompactionFileInfo{
          static_cast<int>(i), file_number, fmd->oldest_blob_file_number});
3067
      if (compaction_job_info->table_properties.count(fn) == 0) {
3068
        std::shared_ptr<const TableProperties> tp;
3069 3070 3071 3072 3073 3074 3075 3076
        auto s = current->GetTableProperties(&tp, fmd, &fn);
        if (s.ok()) {
          compaction_job_info->table_properties[fn] = tp;
        }
      }
    }
  }
  for (const auto& newf : c->edit()->GetNewFiles()) {
3077 3078 3079 3080 3081 3082 3083
    const FileMetaData& meta = newf.second;
    const FileDescriptor& desc = meta.fd;
    const uint64_t file_number = desc.GetNumber();
    compaction_job_info->output_files.push_back(TableFileName(
        c->immutable_cf_options()->cf_paths, file_number, desc.GetPathId()));
    compaction_job_info->output_file_infos.push_back(CompactionFileInfo{
        newf.first, file_number, meta.oldest_blob_file_number});
3084 3085 3086 3087
  }
}
#endif

3088
// SuperVersionContext gets created and destructed outside of the lock --
Y
Yanqin Jin 已提交
3089
// we use this conveniently to:
S
Siying Dong 已提交
3090 3091 3092 3093
// * malloc one SuperVersion() outside of the lock -- new_superversion
// * delete SuperVersion()s outside of the lock -- superversions_to_free
//
// However, if InstallSuperVersionAndScheduleWork() gets called twice with the
3094
// same sv_context, we can't reuse the SuperVersion() that got
S
Siying Dong 已提交
3095 3096 3097 3098 3099
// malloced because
// first call already used it. In that rare case, we take a hit and create a
// new SuperVersion() inside of the mutex. We do similar thing
// for superversion_to_free

3100 3101
void DBImpl::InstallSuperVersionAndScheduleWork(
    ColumnFamilyData* cfd, SuperVersionContext* sv_context,
Y
Yanqin Jin 已提交
3102
    const MutableCFOptions& mutable_cf_options) {
S
Siying Dong 已提交
3103 3104 3105 3106 3107 3108 3109 3110 3111 3112
  mutex_.AssertHeld();

  // Update max_total_in_memory_state_
  size_t old_memtable_size = 0;
  auto* old_sv = cfd->GetSuperVersion();
  if (old_sv) {
    old_memtable_size = old_sv->mutable_cf_options.write_buffer_size *
                        old_sv->mutable_cf_options.max_write_buffer_number;
  }

3113 3114
  // this branch is unlikely to step in
  if (UNLIKELY(sv_context->new_superversion == nullptr)) {
3115 3116
    sv_context->NewSuperVersion();
  }
3117
  cfd->InstallSuperVersion(sv_context, &mutex_, mutable_cf_options);
S
Siying Dong 已提交
3118

3119 3120 3121 3122 3123 3124 3125 3126 3127 3128 3129
  // There may be a small data race here. The snapshot tricking bottommost
  // compaction may already be released here. But assuming there will always be
  // newer snapshot created and released frequently, the compaction will be
  // triggered soon anyway.
  bottommost_files_mark_threshold_ = kMaxSequenceNumber;
  for (auto* my_cfd : *versions_->GetColumnFamilySet()) {
    bottommost_files_mark_threshold_ = std::min(
        bottommost_files_mark_threshold_,
        my_cfd->current()->storage_info()->bottommost_files_mark_threshold());
  }

S
Siying Dong 已提交
3130 3131 3132 3133 3134 3135
  // Whenever we install new SuperVersion, we might need to issue new flushes or
  // compactions.
  SchedulePendingCompaction(cfd);
  MaybeScheduleFlushOrCompaction();

  // Update max_total_in_memory_state_
3136 3137 3138
  max_total_in_memory_state_ = max_total_in_memory_state_ - old_memtable_size +
                               mutable_cf_options.write_buffer_size *
                                   mutable_cf_options.max_write_buffer_number;
S
Siying Dong 已提交
3139
}
Y
Yi Wu 已提交
3140

3141
// ShouldPurge is called by FindObsoleteFiles when doing a full scan,
Y
Yi Wu 已提交
3142
// and db mutex (mutex_) should already be held.
3143 3144 3145 3146
// Actually, the current implementation of FindObsoleteFiles with
// full_scan=true can issue I/O requests to obtain list of files in
// directories, e.g. env_->getChildren while holding db mutex.
bool DBImpl::ShouldPurge(uint64_t file_number) const {
Y
Yi Wu 已提交
3147 3148 3149
  return files_grabbed_for_purge_.find(file_number) ==
             files_grabbed_for_purge_.end() &&
         purge_files_.find(file_number) == purge_files_.end();
3150 3151 3152 3153 3154
}

// MarkAsGrabbedForPurge is called by FindObsoleteFiles, and db mutex
// (mutex_) should already be held.
void DBImpl::MarkAsGrabbedForPurge(uint64_t file_number) {
Y
Yi Wu 已提交
3155
  files_grabbed_for_purge_.insert(file_number);
3156 3157
}

Y
Yi Wu 已提交
3158 3159 3160 3161 3162 3163 3164 3165
void DBImpl::SetSnapshotChecker(SnapshotChecker* snapshot_checker) {
  InstrumentedMutexLock l(&mutex_);
  // snapshot_checker_ should only set once. If we need to set it multiple
  // times, we need to make sure the old one is not deleted while it is still
  // using by a compaction job.
  assert(!snapshot_checker_);
  snapshot_checker_.reset(snapshot_checker);
}
3166 3167 3168 3169 3170 3171 3172 3173 3174 3175 3176 3177 3178 3179 3180 3181 3182 3183 3184 3185 3186 3187 3188 3189 3190 3191 3192

void DBImpl::GetSnapshotContext(
    JobContext* job_context, std::vector<SequenceNumber>* snapshot_seqs,
    SequenceNumber* earliest_write_conflict_snapshot,
    SnapshotChecker** snapshot_checker_ptr) {
  mutex_.AssertHeld();
  assert(job_context != nullptr);
  assert(snapshot_seqs != nullptr);
  assert(earliest_write_conflict_snapshot != nullptr);
  assert(snapshot_checker_ptr != nullptr);

  *snapshot_checker_ptr = snapshot_checker_.get();
  if (use_custom_gc_ && *snapshot_checker_ptr == nullptr) {
    *snapshot_checker_ptr = DisableGCSnapshotChecker::Instance();
  }
  if (*snapshot_checker_ptr != nullptr) {
    // If snapshot_checker is used, that means the flush/compaction may
    // contain values not visible to snapshot taken after
    // flush/compaction job starts. Take a snapshot and it will appear
    // in snapshot_seqs and force compaction iterator to consider such
    // snapshots.
    const Snapshot* job_snapshot =
        GetSnapshotImpl(false /*write_conflict_boundary*/, false /*lock*/);
    job_context->job_snapshot.reset(new ManagedSnapshot(this, job_snapshot));
  }
  *snapshot_seqs = snapshots_.GetAll(earliest_write_conflict_snapshot);
}
3193
}  // namespace ROCKSDB_NAMESPACE