db_impl_compaction_flush.cc 108.7 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 9 10 11 12 13 14 15 16
//
// Copyright (c) 2011 The LevelDB Authors. All rights reserved.
// Use of this source code is governed by a BSD-style license that can be
// found in the LICENSE file. See the AUTHORS file for names of contributors.
#include "db/db_impl.h"

#ifndef __STDC_FORMAT_MACROS
#define __STDC_FORMAT_MACROS
#endif
#include <inttypes.h>

#include "db/builder.h"
17
#include "db/error_handler.h"
18
#include "db/event_helpers.h"
19 20 21 22
#include "monitoring/iostats_context_imp.h"
#include "monitoring/perf_context_imp.h"
#include "monitoring/thread_status_updater.h"
#include "monitoring/thread_status_util.h"
23
#include "util/concurrent_task_limiter_impl.h"
S
Siying Dong 已提交
24 25
#include "util/sst_file_manager_impl.h"
#include "util/sync_point.h"
S
Siying Dong 已提交
26 27

namespace rocksdb {
28

29
bool DBImpl::EnoughRoomForCompaction(
30
    ColumnFamilyData* cfd, const std::vector<CompactionInputFiles>& inputs,
31 32 33 34 35 36 37
    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) {
38 39 40 41 42
    // 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());
43 44 45 46
    if (enough_room) {
      *sfm_reserved_compact_space = true;
    }
  }
47
#else
48
  (void)cfd;
49 50
  (void)inputs;
  (void)sfm_reserved_compact_space;
51 52 53 54 55 56 57 58 59 60 61 62
#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 已提交
63
bool DBImpl::RequestCompactionToken(ColumnFamilyData* cfd, bool force,
64 65 66 67 68 69 70 71 72 73 74
                                    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 已提交
75 76 77 78
                     "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());
79 80 81 82 83
    return true;
  }
  return false;
}

S
Siying Dong 已提交
84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101 102 103 104 105 106 107 108 109
Status DBImpl::SyncClosedLogs(JobContext* job_context) {
  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);
  }

  Status s;
  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());
      s = log->file()->Sync(immutable_db_options_.use_fsync);
110 111 112
      if (!s.ok()) {
        break;
      }
S
Siying Dong 已提交
113 114 115 116 117 118 119 120 121 122 123
    }
    if (s.ok()) {
      s = directories_.GetWalDir()->Fsync();
    }

    mutex_.Lock();

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

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


  FlushJob flush_job(
145
      dbname_, cfd, immutable_db_options_, mutable_cf_options,
146 147 148 149
      nullptr /* memtable_id */, env_options_for_compaction_, versions_.get(),
      &mutex_, &shutting_down_, snapshot_seqs, earliest_write_conflict_snapshot,
      snapshot_checker, job_context, log_buffer, directories_.GetDbDir(),
      GetDataDir(cfd, 0U),
S
Siying Dong 已提交
150
      GetCompressionFlush(*cfd->ioptions(), mutable_cf_options), stats_,
151 152
      &event_logger_, mutable_cf_options.report_bg_io_stats,
      true /* sync_output_directory */, true /* write_manifest */);
S
Siying Dong 已提交
153 154 155

  FileMetaData file_meta;

156
  TEST_SYNC_POINT("DBImpl::FlushMemTableToOutputFile:BeforePickMemtables");
S
Siying Dong 已提交
157
  flush_job.PickMemTable();
158
  TEST_SYNC_POINT("DBImpl::FlushMemTableToOutputFile:AfterPickMemtables");
S
Siying Dong 已提交
159

Y
Yi Wu 已提交
160 161 162 163 164 165
#ifndef ROCKSDB_LITE
  // may temporarily unlock and lock the mutex.
  NotifyOnFlushBegin(cfd, &file_meta, mutable_cf_options, job_context->job_id,
                     flush_job.GetTableProperties());
#endif  // ROCKSDB_LITE

S
Siying Dong 已提交
166 167
  Status s;
  if (logfile_number_ > 0 &&
168
      versions_->GetColumnFamilySet()->NumberOfColumnFamilies() > 1) {
S
Siying Dong 已提交
169 170 171 172 173 174 175
    // 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.
    s = SyncClosedLogs(job_context);
176 177
  } else {
    TEST_SYNC_POINT("DBImpl::SyncClosedLogs:Skip");
S
Siying Dong 已提交
178 179 180 181 182 183 184 185 186
  }

  // 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 已提交
187
    s = flush_job.Run(&logs_with_prep_tracker_, &file_meta);
S
Siying Dong 已提交
188 189 190 191 192
  } else {
    flush_job.Cancel();
  }

  if (s.ok()) {
193 194
    InstallSuperVersionAndScheduleWork(cfd, superversion_context,
                                       mutable_cf_options);
S
Siying Dong 已提交
195
    if (made_progress) {
196
      *made_progress = true;
S
Siying Dong 已提交
197 198 199 200 201 202 203
    }
    VersionStorageInfo::LevelSummaryStorage tmp;
    ROCKS_LOG_BUFFER(log_buffer, "[%s] Level summary: %s\n",
                     cfd->GetName().c_str(),
                     cfd->current()->storage_info()->LevelSummary(&tmp));
  }

204
  if (!s.ok() && !s.IsShutdownInProgress()) {
205
    Status new_bg_error = s;
206
    error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
S
Siying Dong 已提交
207 208 209 210 211 212 213 214 215 216 217
  }
  if (s.ok()) {
#ifndef ROCKSDB_LITE
    // may temporarily unlock and lock the mutex.
    NotifyOnFlushCompleted(cfd, &file_meta, mutable_cf_options,
                           job_context->job_id, flush_job.GetTableProperties());
    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(
218
          cfd->ioptions()->cf_paths[0].path, file_meta.fd.GetNumber());
S
Siying Dong 已提交
219
      sfm->OnAddFile(file_path);
220 221
      if (sfm->IsMaxAllowedSpaceReached()) {
        Status new_bg_error = Status::SpaceLimit("Max allowed space was reached");
S
Siying Dong 已提交
222 223
        TEST_SYNC_POINT_CALLBACK(
            "DBImpl::FlushMemTableToOutputFile:MaxAllowedSpaceReached",
224
            &new_bg_error);
225
        error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
S
Siying Dong 已提交
226 227 228 229 230 231 232
      }
    }
#endif  // ROCKSDB_LITE
  }
  return s;
}

233 234 235
Status DBImpl::FlushMemTablesToOutputFiles(
    const autovector<BGFlushArg>& bg_flush_args, bool* made_progress,
    JobContext* job_context, LogBuffer* log_buffer) {
236
  if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
237 238 239
    return AtomicFlushMemTablesToOutputFiles(bg_flush_args, made_progress,
                                             job_context, log_buffer);
  }
240 241 242 243 244
  std::vector<SequenceNumber> snapshot_seqs;
  SequenceNumber earliest_write_conflict_snapshot;
  SnapshotChecker* snapshot_checker;
  GetSnapshotContext(job_context, &snapshot_seqs,
                     &earliest_write_conflict_snapshot, &snapshot_checker);
245
  Status status;
246 247
  for (auto& arg : bg_flush_args) {
    ColumnFamilyData* cfd = arg.cfd_;
248
    MutableCFOptions mutable_cf_options = *cfd->GetLatestMutableCFOptions();
249
    SuperVersionContext* superversion_context = arg.superversion_context_;
250 251 252 253
    Status s = FlushMemTableToOutputFile(
        cfd, mutable_cf_options, made_progress, job_context,
        superversion_context, snapshot_seqs, earliest_write_conflict_snapshot,
        snapshot_checker, log_buffer);
254
    if (!s.ok()) {
255 256 257 258 259 260
      status = s;
      if (!s.IsShutdownInProgress()) {
        // At this point, DB is not shutting down, nor is cfd dropped.
        // Something is wrong, thus we break out of the loop.
        break;
      }
261 262
    }
  }
263
  return status;
264 265
}

266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282 283 284 285 286 287 288 289 290 291
/*
 * 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,
    JobContext* job_context, LogBuffer* log_buffer) {
  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 */

292
  std::vector<SequenceNumber> snapshot_seqs;
293
  SequenceNumber earliest_write_conflict_snapshot;
294 295 296
  SnapshotChecker* snapshot_checker;
  GetSnapshotContext(job_context, &snapshot_seqs,
                     &earliest_write_conflict_snapshot, &snapshot_checker);
297 298 299

  autovector<Directory*> distinct_output_dirs;
  std::vector<FlushJob> jobs;
300
  std::vector<MutableCFOptions> all_mutable_cf_options;
301
  int num_cfs = static_cast<int>(cfds.size());
302
  all_mutable_cf_options.reserve(num_cfs);
303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320
  for (int i = 0; i < num_cfs; ++i) {
    auto cfd = cfds[i];
    Directory* data_dir = GetDataDir(cfd, 0U);

    // 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;
    for (const auto dir : distinct_output_dirs) {
      if (dir == data_dir) {
        found = true;
        break;
      }
    }
    if (!found) {
      distinct_output_dirs.emplace_back(data_dir);
    }

321 322
    all_mutable_cf_options.emplace_back(*cfd->GetLatestMutableCFOptions());
    const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.back();
323 324 325 326 327 328 329 330 331 332 333 334
    const uint64_t* max_memtable_id = &(bg_flush_args[i].max_memtable_id_);
    jobs.emplace_back(
        dbname_, cfds[i], immutable_db_options_, mutable_cf_options,
        max_memtable_id, env_options_for_compaction_, versions_.get(), &mutex_,
        &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,
        false /* sync_output_directory */, false /* write_manifest */);
    jobs.back().PickMemTable();
  }

335
  std::vector<FileMetaData> file_meta(num_cfs);
336 337 338 339
  Status s;
  assert(num_cfs == static_cast<int>(jobs.size()));

#ifndef ROCKSDB_LITE
340 341
  for (int i = 0; i != num_cfs; ++i) {
    const MutableCFOptions& mutable_cf_options = all_mutable_cf_options.at(i);
342 343 344 345
    // may temporarily unlock and lock the mutex.
    NotifyOnFlushBegin(cfds[i], &file_meta[i], mutable_cf_options,
                       job_context->job_id, jobs[i].GetTableProperties());
  }
346
#endif /* !ROCKSDB_LITE */
347 348 349 350 351 352 353

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

Y
Yanqin Jin 已提交
354 355 356 357 358
  // exec_status stores the execution status of flush_jobs as
  // <bool /* executed */, Status /* status code */>
  autovector<std::pair<bool, Status>> exec_status;
  for (int i = 0; i != num_cfs; ++i) {
    // Initially all jobs are not executed, with status OK.
359
    exec_status.emplace_back(false, Status::OK());
Y
Yanqin Jin 已提交
360 361
  }

362 363
  if (s.ok()) {
    // TODO (yanqin): parallelize jobs with threads.
364
    for (int i = 1; i != num_cfs; ++i) {
Y
Yanqin Jin 已提交
365 366 367
      exec_status[i].second =
          jobs[i].Run(&logs_with_prep_tracker_, &file_meta[i]);
      exec_status[i].first = true;
368
    }
369 370 371 372 373 374
    if (num_cfs > 1) {
      TEST_SYNC_POINT(
          "DBImpl::AtomicFlushMemTablesToOutputFiles:SomeFlushJobsComplete:1");
      TEST_SYNC_POINT(
          "DBImpl::AtomicFlushMemTablesToOutputFiles:SomeFlushJobsComplete:2");
    }
375 376 377 378 379 380 381 382 383 384 385 386 387 388
    exec_status[0].second =
        jobs[0].Run(&logs_with_prep_tracker_, &file_meta[0]);
    exec_status[0].first = true;

    Status error_status;
    for (const auto& e : exec_status) {
      if (!e.second.ok()) {
        s = e.second;
        if (!e.second.IsShutdownInProgress()) {
          // 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;
        }
389 390
      }
    }
391 392

    s = error_status.ok() ? s : error_status;
393 394
  }

395
  if (s.ok() || s.IsShutdownInProgress()) {
396 397 398 399 400 401 402 403 404
    // Sync on all distinct output directories.
    for (auto dir : distinct_output_dirs) {
      if (dir != nullptr) {
        s = dir->Fsync();
        if (!s.ok()) {
          break;
        }
      }
    }
405
  }
406

407 408 409 410
  if (s.ok()) {
    auto wait_to_install_func = [&]() {
      bool ready = true;
      for (size_t i = 0; i != cfds.size(); ++i) {
411
        const auto& mems = jobs[i].GetMemTables();
412 413
        if (cfds[i]->IsDropped()) {
          // If the column family is dropped, then do not wait.
414
          continue;
415 416 417 418 419 420 421 422 423 424 425 426 427 428 429
        } 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;
430
        }
431
      }
432 433 434 435 436 437 438 439 440 441 442 443 444
      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();
  }
445

446 447 448 449
  if (s.ok()) {
    autovector<ColumnFamilyData*> tmp_cfds;
    autovector<const autovector<MemTable*>*> mems_list;
    autovector<const MutableCFOptions*> mutable_cf_options_list;
450
    autovector<FileMetaData*> tmp_file_meta;
451 452 453 454 455
    for (int i = 0; i != num_cfs; ++i) {
      const auto& mems = jobs[i].GetMemTables();
      if (!cfds[i]->IsDropped() && !mems.empty()) {
        tmp_cfds.emplace_back(cfds[i]);
        mems_list.emplace_back(&mems);
456
        mutable_cf_options_list.emplace_back(&all_mutable_cf_options[i]);
457
        tmp_file_meta.emplace_back(&file_meta[i]);
458
      }
459
    }
460 461 462

    s = InstallMemtableAtomicFlushResults(
        nullptr /* imm_lists */, tmp_cfds, mutable_cf_options_list, mems_list,
463 464
        versions_.get(), &mutex_, tmp_file_meta,
        &job_context->memtables_to_free, directories_.GetDbDir(), log_buffer);
465 466
  }

467
  if (s.ok() || s.IsShutdownInProgress()) {
468 469 470
    assert(num_cfs ==
           static_cast<int>(job_context->superversion_contexts.size()));
    for (int i = 0; i != num_cfs; ++i) {
471 472 473
      if (cfds[i]->IsDropped()) {
        continue;
      }
474 475
      InstallSuperVersionAndScheduleWork(cfds[i],
                                         &job_context->superversion_contexts[i],
476
                                         all_mutable_cf_options[i]);
477 478 479 480 481 482 483 484 485 486 487 488
      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());
    for (int i = 0; i != num_cfs; ++i) {
489 490 491
      if (cfds[i]->IsDropped()) {
        continue;
      }
492
      NotifyOnFlushCompleted(cfds[i], &file_meta[i], all_mutable_cf_options[i],
493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509
                             job_context->job_id, jobs[i].GetTableProperties());
      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
  }

510 511 512
  // 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.IsShutdownInProgress()) {
Y
Yanqin Jin 已提交
513 514
    // Have to cancel the flush jobs that have NOT executed because we need to
    // unref the versions.
515
    for (int i = 0; i != num_cfs; ++i) {
Y
Yanqin Jin 已提交
516 517 518
      if (!exec_status[i].first) {
        jobs[i].Cancel();
      }
519
    }
520 521 522 523 524
    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());
Y
Yanqin Jin 已提交
525
      }
526
    }
527 528
    Status new_bg_error = s;
    error_handler_.SetBGError(new_bg_error, BackgroundErrorReason::kFlush);
529 530 531 532 533
  }

  return s;
}

Y
Yi Wu 已提交
534 535 536 537 538 539 540 541 542 543 544 545 546 547 548 549 550 551 552 553 554
void DBImpl::NotifyOnFlushBegin(ColumnFamilyData* cfd, FileMetaData* file_meta,
                                const MutableCFOptions& mutable_cf_options,
                                int job_id, TableProperties prop) {
#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();
  {
    FlushJobInfo info;
555
    info.cf_id = cfd->GetID();
Y
Yi Wu 已提交
556 557 558
    info.cf_name = cfd->GetName();
    // TODO(yhchiang): make db_paths dynamic in case flush does not
    //                 go to L0 in the future.
559
    info.file_path = MakeTableFileName(cfd->ioptions()->cf_paths[0].path,
Y
Yi Wu 已提交
560 561 562 563 564
                                       file_meta->fd.GetNumber());
    info.thread_id = env_->GetThreadID();
    info.job_id = job_id;
    info.triggered_writes_slowdown = triggered_writes_slowdown;
    info.triggered_writes_stop = triggered_writes_stop;
565 566
    info.smallest_seqno = file_meta->fd.smallest_seqno;
    info.largest_seqno = file_meta->fd.largest_seqno;
Y
Yi Wu 已提交
567
    info.table_properties = prop;
568
    info.flush_reason = cfd->GetFlushReason();
Y
Yi Wu 已提交
569 570 571 572 573 574 575
    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.
576 577 578 579 580 581
#else
  (void)cfd;
  (void)file_meta;
  (void)mutable_cf_options;
  (void)job_id;
  (void)prop;
Y
Yi Wu 已提交
582 583 584
#endif  // ROCKSDB_LITE
}

S
Siying Dong 已提交
585 586 587 588 589 590 591 592 593 594 595 596 597 598 599 600 601 602 603 604 605 606
void DBImpl::NotifyOnFlushCompleted(ColumnFamilyData* cfd,
                                    FileMetaData* file_meta,
                                    const MutableCFOptions& mutable_cf_options,
                                    int job_id, TableProperties prop) {
#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();
  {
    FlushJobInfo info;
607
    info.cf_id = cfd->GetID();
S
Siying Dong 已提交
608 609 610
    info.cf_name = cfd->GetName();
    // TODO(yhchiang): make db_paths dynamic in case flush does not
    //                 go to L0 in the future.
611
    info.file_path = MakeTableFileName(cfd->ioptions()->cf_paths[0].path,
S
Siying Dong 已提交
612 613 614 615 616
                                       file_meta->fd.GetNumber());
    info.thread_id = env_->GetThreadID();
    info.job_id = job_id;
    info.triggered_writes_slowdown = triggered_writes_slowdown;
    info.triggered_writes_stop = triggered_writes_stop;
617 618
    info.smallest_seqno = file_meta->fd.smallest_seqno;
    info.largest_seqno = file_meta->fd.largest_seqno;
S
Siying Dong 已提交
619
    info.table_properties = prop;
620
    info.flush_reason = cfd->GetFlushReason();
S
Siying Dong 已提交
621 622 623 624 625 626 627
    for (auto listener : immutable_db_options_.listeners) {
      listener->OnFlushCompleted(this, info);
    }
  }
  mutex_.Lock();
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
628 629 630 631 632 633
#else
  (void)cfd;
  (void)file_meta;
  (void)mutable_cf_options;
  (void)job_id;
  (void)prop;
S
Siying Dong 已提交
634 635 636 637 638 639
#endif  // ROCKSDB_LITE
}

Status DBImpl::CompactRange(const CompactRangeOptions& options,
                            ColumnFamilyHandle* column_family,
                            const Slice* begin, const Slice* end) {
640 641 642 643
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  auto cfd = cfh->cfd();

  if (options.target_path_id >= cfd->ioptions()->cf_paths.size()) {
S
Siying Dong 已提交
644 645 646 647 648
    return Status::InvalidArgument("Invalid target path ID");
  }

  bool exclusive = options.exclusive_manual_compaction;

649
  bool flush_needed = true;
650 651 652 653 654 655 656 657 658 659
  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);
    SuperVersion* super_version = cfd->GetReferencedSuperVersion(&mutex_);
    cfd->RangesOverlapWithMemtables({range}, super_version, &flush_needed);
    CleanupSuperVersion(super_version);
  }

660 661
  Status s;
  if (flush_needed) {
662 663
    FlushOptions fo;
    fo.allow_write_stall = options.allow_write_stall;
664
    if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
665
      autovector<ColumnFamilyData*> cfds;
666
      mutex_.Lock();
Y
Yanqin Jin 已提交
667
      SelectColumnFamiliesForAtomicFlush(&cfds);
668
      mutex_.Unlock();
Y
Yanqin Jin 已提交
669 670 671 672 673 674
      s = AtomicFlushMemTables(cfds, fo, FlushReason::kManualCompaction,
                               false /* writes_stopped */);
    } else {
      s = FlushMemTable(cfd, fo, FlushReason::kManualCompaction,
                        false /* writes_stopped*/);
    }
675 676 677 678
    if (!s.ok()) {
      LogFlush(immutable_db_options_.info_log);
      return s;
    }
S
Siying Dong 已提交
679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696
  }

  int max_level_with_files = 0;
  {
    InstrumentedMutexLock l(&mutex_);
    Version* base = cfd->current();
    for (int level = 1; level < base->storage_info()->num_non_empty_levels();
         level++) {
      if (base->storage_info()->OverlapInLevel(level, begin, end)) {
        max_level_with_files = level;
      }
    }
  }

  int final_output_level = 0;
  if (cfd->ioptions()->compaction_style == kCompactionStyleUniversal &&
      cfd->NumberLevels() > 1) {
    // Always compact all files together.
697 698 699 700 701
    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 已提交
702
    s = RunManualCompaction(cfd, ColumnFamilyData::kCompactAllLevels,
703 704
                            final_output_level, options.target_path_id,
                            options.max_subcompactions, begin, end, exclusive);
S
Siying Dong 已提交
705 706 707 708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737
  } else {
    for (int level = 0; level <= max_level_with_files; level++) {
      int output_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_level_with_files && 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;
      } 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.target_path_id,
738
                              options.max_subcompactions, begin, end, exclusive);
S
Siying Dong 已提交
739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757 758 759 760 761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776
      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");
    }
  }
  if (!s.ok()) {
    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;
}

777 778 779 780
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,
781 782
                            std::vector<std::string>* const output_file_names,
                            CompactionJobInfo* compaction_job_info) {
S
Siying Dong 已提交
783
#ifdef ROCKSDB_LITE
784 785 786 787 788 789
  (void)compact_options;
  (void)column_family;
  (void)input_file_names;
  (void)output_level;
  (void)output_path_id;
  (void)output_file_names;
790
  (void)compaction_job_info;
791
  // not supported in lite version
S
Siying Dong 已提交
792 793 794 795 796 797 798 799 800 801 802 803 804 805 806
  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 已提交
807
  TEST_SYNC_POINT("TestCompactFiles::IngestExternalFile2");
S
Siying Dong 已提交
808 809 810 811 812 813 814
  {
    InstrumentedMutexLock l(&mutex_);

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

D
DorianZheng 已提交
815 816 817 818 819 820
    // 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,
821
                         output_file_names, output_level, output_path_id,
822
                         &job_context, &log_buffer, compaction_job_info);
D
DorianZheng 已提交
823 824

    current->Unref();
S
Siying Dong 已提交
825 826 827 828 829 830 831 832 833 834 835 836 837
  }

  // 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
838 839
  if (job_context.HaveSomethingToClean() ||
      job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
S
Siying Dong 已提交
840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856 857 858 859 860
    // 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,
861
    std::vector<std::string>* const output_file_names, const int output_level,
862 863
    int output_path_id, JobContext* job_context, LogBuffer* log_buffer,
    CompactionJobInfo* compaction_job_info) {
S
Siying Dong 已提交
864 865 866 867 868 869 870
  mutex_.AssertHeld();

  if (shutting_down_.load(std::memory_order_acquire)) {
    return Status::ShutdownInProgress();
  }

  std::unordered_set<uint64_t> input_set;
871
  for (const auto& file_name : input_file_names) {
S
Siying Dong 已提交
872 873 874 875 876 877 878 879 880
    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) {
881
    if (cfd->ioptions()->cf_paths.size() == 1U) {
S
Siying Dong 已提交
882 883 884 885 886 887 888 889 890 891 892 893 894 895 896 897 898 899 900 901 902
      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;
  }

903
  for (const auto& inputs : input_files) {
S
Siying Dong 已提交
904
    if (cfd->compaction_picker()->AreFilesInCompaction(inputs.files)) {
S
Siying Dong 已提交
905 906 907 908 909
      return Status::Aborted(
          "Some of the necessary compaction input "
          "files are already being compacted");
    }
  }
910 911 912
  bool sfm_reserved_compact_space = false;
  // First check if we have enough room to do the compaction
  bool enough_room = EnoughRoomForCompaction(
913
      cfd, input_files, &sfm_reserved_compact_space, log_buffer);
914 915 916 917 918 919

  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 已提交
920 921 922 923

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

924
  std::unique_ptr<Compaction> c;
S
Siying Dong 已提交
925
  assert(cfd->compaction_picker());
S
Siying Dong 已提交
926
  c.reset(cfd->compaction_picker()->CompactFiles(
S
Siying Dong 已提交
927 928
      compact_options, input_files, output_level, version->storage_info(),
      *cfd->GetLatestMutableCFOptions(), output_path_id));
929 930 931 932
  // 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 已提交
933 934 935 936
  c->SetInputVersion(version);
  // deletion compaction currently not allowed in CompactFiles.
  assert(!c->deletion_compaction());

937
  std::vector<SequenceNumber> snapshot_seqs;
S
Siying Dong 已提交
938
  SequenceNumber earliest_write_conflict_snapshot;
939 940 941
  SnapshotChecker* snapshot_checker;
  GetSnapshotContext(job_context, &snapshot_seqs,
                     &earliest_write_conflict_snapshot, &snapshot_checker);
S
Siying Dong 已提交
942 943 944 945 946

  auto pending_outputs_inserted_elem =
      CaptureCurrentFileNumberInPendingOutputs();

  assert(is_snapshot_supported_ || snapshots_.empty());
947
  CompactionJobStats compaction_job_stats;
S
Siying Dong 已提交
948
  CompactionJob compaction_job(
949
      job_context->job_id, c.get(), immutable_db_options_,
950 951
      env_options_for_compaction_, versions_.get(), &shutting_down_,
      preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
952
      GetDataDir(c->column_family_data(), c->output_path_id()), stats_, &mutex_,
953
      &error_handler_, snapshot_seqs, earliest_write_conflict_snapshot,
954
      snapshot_checker, table_cache_, &event_logger_,
955
      c->mutable_cf_options()->paranoid_file_checks,
S
Siying Dong 已提交
956
      c->mutable_cf_options()->report_bg_io_stats, dbname_,
957
      &compaction_job_stats);
S
Siying Dong 已提交
958 959 960 961 962 963 964 965 966 967 968 969 970 971 972 973 974 975 976 977

  // 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 已提交
978 979 980
    InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                       &job_context->superversion_contexts[0],
                                       *c->mutable_cf_options());
S
Siying Dong 已提交
981 982
  }
  c->ReleaseCompactionFiles(s);
983 984 985 986 987 988 989 990
#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 已提交
991 992 993

  ReleaseFileNumberFromPendingOutputs(pending_outputs_inserted_elem);

994 995 996 997 998
  if (compaction_job_info != nullptr) {
    BuildCompactionJobInfo(cfd, c.get(), s, compaction_job_stats,
                           job_context->job_id, version, compaction_job_info);
  }

S
Siying Dong 已提交
999 1000 1001 1002 1003 1004 1005 1006 1007
  if (status.ok()) {
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } 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());
1008
    error_handler_.SetBGError(status, BackgroundErrorReason::kCompaction);
S
Siying Dong 已提交
1009 1010
  }

1011 1012
  if (output_file_names != nullptr) {
    for (const auto newf : c->edit()->GetNewFiles()) {
1013 1014 1015 1016
      (*output_file_names)
          .push_back(TableFileName(c->immutable_cf_options()->cf_paths,
                                   newf.second.fd.GetNumber(),
                                   newf.second.fd.GetPathId()));
1017 1018 1019
    }
  }

S
Siying Dong 已提交
1020 1021 1022 1023 1024 1025
  c.reset();

  bg_compaction_scheduled_--;
  if (bg_compaction_scheduled_ == 0) {
    bg_cv_.SignalAll();
  }
1026
  TEST_SYNC_POINT("CompactFilesImpl:End");
S
Siying Dong 已提交
1027 1028 1029 1030 1031 1032 1033 1034

  return status;
}
#endif  // ROCKSDB_LITE

Status DBImpl::PauseBackgroundWork() {
  InstrumentedMutexLock guard_lock(&mutex_);
  bg_compaction_paused_++;
1035 1036
  while (bg_bottom_compaction_scheduled_ > 0 || bg_compaction_scheduled_ > 0 ||
         bg_flush_scheduled_ > 0) {
S
Siying Dong 已提交
1037 1038 1039 1040 1041 1042 1043 1044 1045 1046 1047 1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059
    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();
}

P
Peter Pei 已提交
1060 1061 1062 1063 1064 1065 1066 1067 1068 1069 1070 1071 1072 1073 1074 1075 1076 1077 1078 1079 1080 1081 1082 1083 1084 1085 1086 1087 1088 1089 1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103 1104 1105 1106 1107 1108 1109 1110 1111 1112 1113 1114 1115 1116 1117 1118 1119 1120 1121 1122
void DBImpl::NotifyOnCompactionBegin(ColumnFamilyData* cfd,
                                     Compaction *c, const Status &st,
                                     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;
  }
  Version* current = cfd->current();
  current->Ref();
  // release lock while notifying events
  mutex_.Unlock();
  TEST_SYNC_POINT("DBImpl::NotifyOnCompactionBegin::UnlockMutex");
  {
    CompactionJobInfo info;
    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)) {
        auto fn = TableFileName(c->immutable_cf_options()->cf_paths,
                                fmd->fd.GetNumber(), fmd->fd.GetPathId());
        info.input_files.push_back(fn);
        if (info.table_properties.count(fn) == 0) {
          std::shared_ptr<const TableProperties> tp;
          auto s = current->GetTableProperties(&tp, fmd, &fn);
          if (s.ok()) {
            info.table_properties[fn] = tp;
          }
        }
      }
    }
    for (const auto newf : c->edit()->GetNewFiles()) {
      info.output_files.push_back(TableFileName(
          c->immutable_cf_options()->cf_paths, newf.second.fd.GetNumber(),
          newf.second.fd.GetPathId()));
    }
    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 已提交
1123
void DBImpl::NotifyOnCompactionCompleted(
1124 1125
    ColumnFamilyData* cfd, Compaction* c, const Status& st,
    const CompactionJobStats& compaction_job_stats, const int job_id) {
S
Siying Dong 已提交
1126 1127 1128 1129 1130 1131 1132 1133
#ifndef ROCKSDB_LITE
  if (immutable_db_options_.listeners.size() == 0U) {
    return;
  }
  mutex_.AssertHeld();
  if (shutting_down_.load(std::memory_order_acquire)) {
    return;
  }
1134 1135
  Version* current = cfd->current();
  current->Ref();
S
Siying Dong 已提交
1136 1137 1138 1139 1140
  // release lock while notifying events
  mutex_.Unlock();
  TEST_SYNC_POINT("DBImpl::NotifyOnCompactionCompleted::UnlockMutex");
  {
    CompactionJobInfo info;
1141 1142
    BuildCompactionJobInfo(cfd, c, st, compaction_job_stats, job_id, current,
                           &info);
S
Siying Dong 已提交
1143 1144 1145 1146 1147
    for (auto listener : immutable_db_options_.listeners) {
      listener->OnCompactionCompleted(this, info);
    }
  }
  mutex_.Lock();
1148
  current->Unref();
S
Siying Dong 已提交
1149 1150
  // no need to signal bg_cv_ as it will be signaled at the end of the
  // flush process.
1151 1152 1153 1154 1155 1156
#else
  (void)cfd;
  (void)c;
  (void)st;
  (void)compaction_job_stats;
  (void)job_id;
S
Siying Dong 已提交
1157 1158 1159 1160 1161 1162 1163 1164 1165 1166 1167
#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");
  }

1168
  SuperVersionContext sv_context(/* create_superversion */ true);
S
Siying Dong 已提交
1169 1170 1171 1172 1173 1174 1175 1176 1177 1178 1179 1180 1181 1182 1183 1184 1185 1186 1187 1188 1189 1190 1191 1192 1193 1194 1195 1196 1197 1198 1199 1200 1201 1202 1203 1204 1205 1206 1207 1208 1209 1210 1211 1212 1213

  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,
1214
                   f->fd.smallest_seqno, f->fd.largest_seqno,
S
Siying Dong 已提交
1215 1216 1217 1218 1219 1220 1221 1222
                   f->marked_for_compaction);
    }
    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());
1223
    InstallSuperVersionAndScheduleWork(cfd, &sv_context, mutable_cf_options);
S
Siying Dong 已提交
1224 1225 1226 1227 1228 1229 1230 1231 1232 1233 1234

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

1235
  sv_context.Clean();
S
Siying Dong 已提交
1236 1237 1238 1239 1240 1241 1242 1243 1244 1245
  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 已提交
1246
int DBImpl::MaxMemCompactionLevel(ColumnFamilyHandle* /*column_family*/) {
S
Siying Dong 已提交
1247 1248 1249 1250 1251 1252
  return 0;
}

int DBImpl::Level0StopWriteTrigger(ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
  InstrumentedMutexLock l(&mutex_);
1253 1254 1255
  return cfh->cfd()
      ->GetSuperVersion()
      ->mutable_cf_options.level0_stop_writes_trigger;
S
Siying Dong 已提交
1256 1257 1258 1259 1260
}

Status DBImpl::Flush(const FlushOptions& flush_options,
                     ColumnFamilyHandle* column_family) {
  auto cfh = reinterpret_cast<ColumnFamilyHandleImpl*>(column_family);
1261 1262
  ROCKS_LOG_INFO(immutable_db_options_.info_log, "[%s] Manual flush start.",
                 cfh->GetName().c_str());
Y
Yanqin Jin 已提交
1263
  Status s;
1264
  if (immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1265 1266 1267 1268 1269 1270
    s = AtomicFlushMemTables({cfh->cfd()}, flush_options,
                             FlushReason::kManualFlush);
  } else {
    s = FlushMemTable(cfh->cfd(), flush_options, FlushReason::kManualFlush);
  }

1271 1272 1273 1274
  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 已提交
1275 1276
}

Y
Yanqin Jin 已提交
1277 1278
Status DBImpl::Flush(const FlushOptions& flush_options,
                     const std::vector<ColumnFamilyHandle*>& column_families) {
1279
  Status s;
1280
  if (!immutable_db_options_.atomic_flush) {
Y
Yanqin Jin 已提交
1281 1282 1283 1284 1285
    for (auto cfh : column_families) {
      s = Flush(flush_options, cfh);
      if (!s.ok()) {
        break;
      }
1286
    }
Y
Yanqin Jin 已提交
1287 1288 1289 1290 1291 1292 1293 1294
  } 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());
1295
    }
Y
Yanqin Jin 已提交
1296 1297 1298 1299 1300 1301 1302 1303 1304 1305 1306 1307 1308 1309 1310 1311
    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,
                   "Manual atomic flush finished, status: %s\n",
                   "=====Column families:=====", s.ToString().c_str());
    for (auto cfh : column_families) {
      auto cfhi = static_cast<ColumnFamilyHandleImpl*>(cfh);
      ROCKS_LOG_INFO(immutable_db_options_.info_log, "%s",
                     cfhi->GetName().c_str());
1312
    }
Y
Yanqin Jin 已提交
1313 1314
    ROCKS_LOG_INFO(immutable_db_options_.info_log,
                   "=====End of column families list=====");
1315 1316 1317 1318
  }
  return s;
}

S
Siying Dong 已提交
1319 1320
Status DBImpl::RunManualCompaction(ColumnFamilyData* cfd, int input_level,
                                   int output_level, uint32_t output_path_id,
1321
                                   uint32_t max_subcompactions,
S
Siying Dong 已提交
1322 1323 1324 1325 1326 1327 1328 1329 1330 1331
                                   const Slice* begin, const Slice* end,
                                   bool exclusive, bool disallow_trivial_move) {
  assert(input_level == ColumnFamilyData::kCompactAllLevels ||
         input_level >= 0);

  InternalKey begin_storage, end_storage;
  CompactionArg* ca;

  bool scheduled = false;
  bool manual_conflict = false;
1332
  ManualCompactionState manual;
S
Siying Dong 已提交
1333 1334 1335 1336 1337 1338 1339 1340 1341 1342 1343 1344 1345 1346 1347 1348
  manual.cfd = cfd;
  manual.input_level = input_level;
  manual.output_level = output_level;
  manual.output_path_id = output_path_id;
  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 已提交
1349
    begin_storage.SetMinPossibleForUserKey(*begin);
S
Siying Dong 已提交
1350 1351 1352 1353 1354 1355 1356
    manual.begin = &begin_storage;
  }
  if (end == nullptr ||
      cfd->ioptions()->compaction_style == kCompactionStyleUniversal ||
      cfd->ioptions()->compaction_style == kCompactionStyleFIFO) {
    manual.end = nullptr;
  } else {
A
Amy Xu 已提交
1357
    end_storage.SetMaxPossibleForUserKey(*end);
S
Siying Dong 已提交
1358 1359 1360 1361 1362 1363 1364 1365 1366 1367 1368 1369 1370 1371 1372 1373 1374 1375 1376 1377 1378 1379 1380 1381
    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) {
1382 1383
    while (bg_bottom_compaction_scheduled_ > 0 ||
           bg_compaction_scheduled_ > 0) {
1384
      TEST_SYNC_POINT("DBImpl::RunManualCompaction:WaitScheduled");
S
Siying Dong 已提交
1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396
      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());

1397 1398
  LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL,
                       immutable_db_options_.info_log.get());
S
Siying Dong 已提交
1399 1400 1401 1402 1403 1404
  // 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;
1405
    Compaction* compaction = nullptr;
S
Siying Dong 已提交
1406 1407
    if (ShouldntRunManualCompaction(&manual) || (manual.in_progress == true) ||
        scheduled ||
1408
        (((manual.manual_end = &manual.tmp_storage1) != nullptr) &&
1409 1410
         ((compaction = manual.cfd->CompactRange(
               *manual.cfd->GetLatestMutableCFOptions(), manual.input_level,
1411 1412 1413
               manual.output_level, manual.output_path_id, max_subcompactions,
               manual.begin, manual.end, &manual.manual_end,
               &manual_conflict)) == nullptr &&
1414
          manual_conflict))) {
S
Siying Dong 已提交
1415 1416 1417 1418 1419 1420 1421 1422 1423 1424 1425
      // 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) {
1426
      if (compaction == nullptr) {
S
Siying Dong 已提交
1427 1428 1429 1430 1431 1432
        manual.done = true;
        bg_cv_.SignalAll();
        continue;
      }
      ca = new CompactionArg;
      ca->db = this;
1433 1434 1435
      ca->prepicked_compaction = new PrepickedCompaction;
      ca->prepicked_compaction->manual_compaction_state = &manual;
      ca->prepicked_compaction->compaction = compaction;
Y
Yanqin Jin 已提交
1436 1437 1438 1439
      if (!RequestCompactionToken(
              cfd, true, &ca->prepicked_compaction->task_token, &log_buffer)) {
        // Don't throttle manual compaction, only count outstanding tasks.
        assert(false);
1440
      }
S
Siying Dong 已提交
1441 1442 1443 1444 1445 1446 1447 1448
      manual.incomplete = false;
      bg_compaction_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this,
                     &DBImpl::UnscheduleCallback);
      scheduled = true;
    }
  }

1449
  log_buffer.FlushBufferToLog();
S
Siying Dong 已提交
1450 1451 1452 1453 1454 1455 1456
  assert(!manual.in_progress);
  assert(HasPendingManualCompaction());
  RemoveManualCompaction(&manual);
  bg_cv_.SignalAll();
  return manual.status;
}

Y
Yanqin Jin 已提交
1457 1458 1459
void DBImpl::GenerateFlushRequest(const autovector<ColumnFamilyData*>& cfds,
                                  FlushRequest* req) {
  assert(req != nullptr);
1460
  req->reserve(cfds.size());
Y
Yanqin Jin 已提交
1461 1462 1463 1464 1465 1466 1467 1468 1469 1470
  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 已提交
1471 1472
Status DBImpl::FlushMemTable(ColumnFamilyData* cfd,
                             const FlushOptions& flush_options,
1473
                             FlushReason flush_reason, bool writes_stopped) {
S
Siying Dong 已提交
1474
  Status s;
1475
  uint64_t flush_memtable_id = 0;
1476 1477 1478 1479 1480 1481 1482 1483
  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;
    }
  }
1484
  FlushRequest flush_req;
S
Siying Dong 已提交
1485 1486 1487 1488 1489 1490 1491 1492 1493
  {
    WriteContext context;
    InstrumentedMutexLock guard_lock(&mutex_);

    WriteThread::Writer w;
    if (!writes_stopped) {
      write_thread_.EnterUnbatched(&w, &mutex_);
    }

1494
    if (!cfd->mem()->IsEmpty() || !cached_recoverable_state_empty_.load()) {
1495
      s = SwitchMemtable(cfd, &context);
1496 1497 1498 1499 1500 1501 1502 1503
    }

    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);
      }
1504 1505 1506 1507 1508 1509 1510 1511 1512 1513
    }

    if (s.ok() && !flush_req.empty()) {
      for (auto& elem : flush_req) {
        ColumnFamilyData* loop_cfd = elem.first;
        loop_cfd->imm()->FlushRequested();
      }
      SchedulePendingFlush(flush_req, flush_reason);
      MaybeScheduleFlushOrCompaction();
    }
S
Siying Dong 已提交
1514 1515 1516 1517 1518 1519 1520

    if (!writes_stopped) {
      write_thread_.ExitUnbatched(&w);
    }
  }

  if (s.ok() && flush_options.wait) {
1521 1522 1523 1524 1525 1526
    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 已提交
1527 1528
    s = WaitForFlushMemTables(cfds, flush_memtable_ids,
                              (flush_reason == FlushReason::kErrorRecovery));
S
Siying Dong 已提交
1529
  }
1530
  TEST_SYNC_POINT("FlushMemTableFinished");
S
Siying Dong 已提交
1531 1532 1533
  return s;
}

Y
Yanqin Jin 已提交
1534 1535 1536 1537 1538 1539 1540 1541 1542 1543 1544 1545 1546 1547 1548 1549 1550 1551 1552 1553 1554 1555 1556 1557 1558 1559 1560 1561 1562 1563 1564 1565 1566 1567 1568 1569 1570 1571 1572 1573 1574 1575 1576
// Flush all elments in 'column_family_datas'
// 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;
    if (!writes_stopped) {
      write_thread_.EnterUnbatched(&w, &mutex_);
    }

    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) {
1577 1578 1579
      if (cfd->mem()->IsEmpty() && cached_recoverable_state_empty_.load()) {
        continue;
      }
Y
Yanqin Jin 已提交
1580 1581 1582 1583 1584 1585 1586 1587 1588 1589 1590 1591 1592 1593 1594 1595 1596 1597 1598 1599 1600
      cfd->Ref();
      s = SwitchMemtable(cfd, &context);
      cfd->Unref();
      if (!s.ok()) {
        break;
      }
    }
    if (s.ok()) {
      AssignAtomicFlushSeq(cfds);
      for (auto cfd : cfds) {
        cfd->imm()->FlushRequested();
      }
      GenerateFlushRequest(cfds, &flush_req);
      SchedulePendingFlush(flush_req, flush_reason);
      MaybeScheduleFlushOrCompaction();
    }

    if (!writes_stopped) {
      write_thread_.ExitUnbatched(&w);
    }
  }
1601
  TEST_SYNC_POINT("DBImpl::AtomicFlushMemTables:AfterScheduleFlush");
Y
Yanqin Jin 已提交
1602 1603 1604 1605 1606 1607 1608 1609 1610 1611 1612 1613

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

1614 1615 1616 1617 1618 1619 1620 1621 1622 1623 1624 1625 1626 1627 1628
// 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,
    bool* flush_needed) {
  {
    *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) {
1629 1630 1631 1632 1633 1634 1635 1636
        // 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();
        }

1637 1638 1639 1640 1641 1642 1643 1644 1645 1646 1647 1648 1649 1650 1651 1652 1653 1654 1655 1656 1657 1658 1659 1660 1661 1662 1663 1664 1665 1666 1667 1668 1669 1670 1671 1672 1673 1674 1675 1676 1677 1678 1679 1680 1681 1682 1683 1684
        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();
      }
      if (cfd->IsDropped() || shutting_down_.load(std::memory_order_acquire)) {
        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();
}

1685 1686 1687 1688 1689 1690 1691 1692
// 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 已提交
1693 1694
// resuming_from_bg_err indicates whether the caller is trying to resume from
// background error or in normal processing.
1695 1696
Status DBImpl::WaitForFlushMemTables(
    const autovector<ColumnFamilyData*>& cfds,
Y
Yanqin Jin 已提交
1697 1698
    const autovector<const uint64_t*>& flush_memtable_ids,
    bool resuming_from_bg_err) {
1699
  int num = static_cast<int>(cfds.size());
S
Siying Dong 已提交
1700 1701
  // Wait until the compaction completes
  InstrumentedMutexLock l(&mutex_);
Y
Yanqin Jin 已提交
1702 1703 1704
  // 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 已提交
1705 1706 1707
    if (shutting_down_.load(std::memory_order_acquire)) {
      return Status::ShutdownInProgress();
    }
Y
Yanqin Jin 已提交
1708 1709 1710 1711
    // If an error has occurred during resumption, then no need to wait.
    if (!error_handler_.GetRecoveryError().ok()) {
      break;
    }
1712 1713 1714 1715 1716 1717 1718 1719 1720 1721 1722 1723 1724 1725 1726
    // 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) {
S
Siying Dong 已提交
1727 1728
      return Status::InvalidArgument("Cannot flush a dropped CF");
    }
1729 1730 1731 1732 1733
    // 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 已提交
1734 1735
    bg_cv_.Wait();
  }
1736
  Status s;
Y
Yanqin Jin 已提交
1737 1738 1739
  // 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()) {
1740
    s = error_handler_.GetBGError();
S
Siying Dong 已提交
1741 1742 1743 1744 1745 1746 1747 1748 1749 1750 1751 1752 1753 1754 1755 1756 1757 1758 1759 1760 1761 1762 1763 1764 1765 1766 1767
  }
  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;
}

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;
1768 1769 1770 1771 1772 1773
  } else if (error_handler_.IsBGWorkStopped() &&
      !error_handler_.IsRecoveryInProgress()) {
    // 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 已提交
1774 1775 1776 1777
  } else if (shutting_down_.load(std::memory_order_acquire)) {
    // DB is being deleted; no more background compactions
    return;
  }
1778
  auto bg_job_limits = GetBGJobLimits();
1779
  bool is_flush_pool_empty =
1780
      env_->GetBackgroundThreads(Env::Priority::HIGH) == 0;
1781
  while (!is_flush_pool_empty && unscheduled_flushes_ > 0 &&
1782
         bg_flush_scheduled_ < bg_job_limits.max_flushes) {
S
Siying Dong 已提交
1783 1784 1785 1786
    bg_flush_scheduled_++;
    env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::HIGH, this);
  }

1787 1788 1789
  // 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 已提交
1790 1791
    while (unscheduled_flushes_ > 0 &&
           bg_flush_scheduled_ + bg_compaction_scheduled_ <
1792
               bg_job_limits.max_flushes) {
S
Siying Dong 已提交
1793 1794 1795 1796 1797 1798 1799 1800
      bg_flush_scheduled_++;
      env_->Schedule(&DBImpl::BGWorkFlush, this, Env::Priority::LOW, this);
    }
  }

  if (bg_compaction_paused_ > 0) {
    // we paused the background compaction
    return;
1801 1802 1803 1804 1805 1806
  } 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 已提交
1807 1808 1809 1810 1811
  }

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

1816
  while (bg_compaction_scheduled_ < bg_job_limits.max_compactions &&
S
Siying Dong 已提交
1817 1818 1819
         unscheduled_compactions_ > 0) {
    CompactionArg* ca = new CompactionArg;
    ca->db = this;
1820
    ca->prepicked_compaction = nullptr;
S
Siying Dong 已提交
1821 1822 1823 1824 1825 1826 1827
    bg_compaction_scheduled_++;
    unscheduled_compactions_--;
    env_->Schedule(&DBImpl::BGWorkCompaction, ca, Env::Priority::LOW, this,
                   &DBImpl::UnscheduleCallback);
  }
}

1828
DBImpl::BGJobLimits DBImpl::GetBGJobLimits() const {
S
Siying Dong 已提交
1829
  mutex_.AssertHeld();
1830 1831 1832 1833 1834 1835 1836 1837 1838 1839 1840 1841 1842 1843 1844 1845
  return GetBGJobLimits(immutable_db_options_.max_background_flushes,
                        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 已提交
1846
  } else {
1847 1848 1849 1850 1851 1852 1853 1854
    // 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 已提交
1855
  }
1856
  return res;
S
Siying Dong 已提交
1857 1858 1859
}

void DBImpl::AddToCompactionQueue(ColumnFamilyData* cfd) {
1860
  assert(!cfd->queued_for_compaction());
S
Siying Dong 已提交
1861 1862
  cfd->Ref();
  compaction_queue_.push_back(cfd);
1863
  cfd->set_queued_for_compaction(true);
S
Siying Dong 已提交
1864 1865 1866 1867 1868 1869
}

ColumnFamilyData* DBImpl::PopFirstFromCompactionQueue() {
  assert(!compaction_queue_.empty());
  auto cfd = *compaction_queue_.begin();
  compaction_queue_.pop_front();
1870 1871
  assert(cfd->queued_for_compaction());
  cfd->set_queued_for_compaction(false);
S
Siying Dong 已提交
1872 1873 1874
  return cfd;
}

1875
DBImpl::FlushRequest DBImpl::PopFirstFromFlushQueue() {
S
Siying Dong 已提交
1876
  assert(!flush_queue_.empty());
1877 1878 1879
  FlushRequest flush_req = flush_queue_.front();
  assert(unscheduled_flushes_ >= static_cast<int>(flush_req.size()));
  unscheduled_flushes_ -= static_cast<int>(flush_req.size());
S
Siying Dong 已提交
1880
  flush_queue_.pop_front();
1881
  // TODO: need to unset flush reason?
1882
  return flush_req;
S
Siying Dong 已提交
1883 1884
}

1885 1886 1887 1888 1889 1890 1891 1892 1893 1894 1895 1896 1897 1898 1899 1900 1901 1902 1903
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 已提交
1904 1905
  for (auto iter = throttled_candidates.rbegin();
       iter != throttled_candidates.rend(); ++iter) {
1906 1907 1908 1909 1910
    compaction_queue_.push_front(*iter);
  }
  return cfd;
}

1911
void DBImpl::SchedulePendingFlush(const FlushRequest& flush_req,
1912
                                  FlushReason flush_reason) {
1913 1914 1915 1916 1917 1918 1919
  if (flush_req.empty()) {
    return;
  }
  for (auto& iter : flush_req) {
    ColumnFamilyData* cfd = iter.first;
    cfd->Ref();
    cfd->SetFlushReason(flush_reason);
S
Siying Dong 已提交
1920
  }
1921 1922
  unscheduled_flushes_ += static_cast<int>(flush_req.size());
  flush_queue_.push_back(flush_req);
S
Siying Dong 已提交
1923 1924 1925
}

void DBImpl::SchedulePendingCompaction(ColumnFamilyData* cfd) {
1926
  if (!cfd->queued_for_compaction() && cfd->NeedsCompaction()) {
S
Siying Dong 已提交
1927 1928 1929 1930 1931
    AddToCompactionQueue(cfd);
    ++unscheduled_compactions_;
  }
}

1932 1933
void DBImpl::SchedulePendingPurge(std::string fname, std::string dir_to_sync,
                                  FileType type, uint64_t number, int job_id) {
S
Siying Dong 已提交
1934
  mutex_.AssertHeld();
1935
  PurgeFileInfo file_info(fname, dir_to_sync, type, number, job_id);
S
Siying Dong 已提交
1936 1937 1938 1939 1940 1941 1942 1943 1944 1945 1946 1947 1948 1949 1950
  purge_queue_.push_back(std::move(file_info));
}

void DBImpl::BGWorkFlush(void* db) {
  IOSTATS_SET_THREAD_POOL_ID(Env::Priority::HIGH);
  TEST_SYNC_POINT("DBImpl::BGWorkFlush");
  reinterpret_cast<DBImpl*>(db)->BackgroundCallFlush();
  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");
1951 1952 1953 1954 1955 1956 1957 1958 1959 1960 1961 1962 1963 1964 1965 1966 1967
  auto prepicked_compaction =
      static_cast<PrepickedCompaction*>(ca.prepicked_compaction);
  reinterpret_cast<DBImpl*>(ca.db)->BackgroundCallCompaction(
      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 已提交
1968 1969 1970 1971 1972 1973 1974 1975 1976 1977 1978 1979
}

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

void DBImpl::UnscheduleCallback(void* arg) {
  CompactionArg ca = *(reinterpret_cast<CompactionArg*>(arg));
  delete reinterpret_cast<CompactionArg*>(arg);
1980 1981 1982 1983 1984
  if (ca.prepicked_compaction != nullptr) {
    if (ca.prepicked_compaction->compaction != nullptr) {
      delete ca.prepicked_compaction->compaction;
    }
    delete ca.prepicked_compaction;
S
Siying Dong 已提交
1985 1986 1987 1988 1989
  }
  TEST_SYNC_POINT("DBImpl::UnscheduleCallback");
}

Status DBImpl::BackgroundFlush(bool* made_progress, JobContext* job_context,
1990
                               LogBuffer* log_buffer, FlushReason* reason) {
S
Siying Dong 已提交
1991 1992
  mutex_.AssertHeld();

1993
  Status status;
1994 1995 1996
  *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
1997 1998 1999 2000
  if (!error_handler_.IsBGWorkStopped()) {
    if (shutting_down_.load(std::memory_order_acquire)) {
      status = Status::ShutdownInProgress();
    }
2001
  } else if (!error_handler_.IsRecoveryInProgress()) {
2002
    status = error_handler_.GetBGError();
S
Siying Dong 已提交
2003 2004 2005 2006 2007 2008
  }

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

2009 2010 2011
  autovector<BGFlushArg> bg_flush_args;
  std::vector<SuperVersionContext>& superversion_contexts =
      job_context->superversion_contexts;
S
Siying Dong 已提交
2012 2013
  while (!flush_queue_.empty()) {
    // This cfd is already referenced
2014 2015 2016 2017 2018 2019 2020 2021 2022 2023 2024 2025
    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
        if (cfd->Unref()) {
          delete cfd;
        }
        continue;
S
Siying Dong 已提交
2026
      }
2027 2028 2029 2030 2031 2032
      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 已提交
2033 2034 2035
    }
  }

2036
  if (!bg_flush_args.empty()) {
2037
    auto bg_job_limits = GetBGJobLimits();
2038 2039 2040 2041 2042 2043 2044 2045 2046 2047 2048 2049 2050 2051
    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,
                                         job_context, log_buffer);
2052 2053 2054
    // 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();
2055 2056 2057 2058 2059 2060
    for (auto& arg : bg_flush_args) {
      ColumnFamilyData* cfd = arg.cfd_;
      if (cfd->Unref()) {
        delete cfd;
        arg.cfd_ = nullptr;
      }
S
Siying Dong 已提交
2061 2062 2063 2064 2065 2066 2067 2068 2069 2070 2071 2072 2073 2074 2075
    }
  }
  return status;
}

void DBImpl::BackgroundCallFlush() {
  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 已提交
2076
    assert(bg_flush_scheduled_);
S
Siying Dong 已提交
2077 2078 2079 2080
    num_running_flushes_++;

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();
2081
    FlushReason reason;
S
Siying Dong 已提交
2082

2083 2084 2085 2086
    Status s =
        BackgroundFlush(&made_progress, &job_context, &log_buffer, &reason);
    if (!s.ok() && !s.IsShutdownInProgress() &&
        reason != FlushReason::kErrorRecovery) {
S
Siying Dong 已提交
2087 2088 2089 2090 2091
      // 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 =
2092
          default_cf_internal_stats_->BumpAndGetBackgroundErrorCount();
S
Siying Dong 已提交
2093 2094 2095 2096 2097 2098 2099 2100 2101 2102 2103 2104
      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 已提交
2105
    TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:FlushFinish:0");
S
Siying Dong 已提交
2106 2107 2108 2109 2110 2111
    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()
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
    // delete unnecessary files if any, this is done outside the mutex
2112 2113
    if (job_context.HaveSomethingToClean() ||
        job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
S
Siying Dong 已提交
2114
      mutex_.Unlock();
2115
      TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:FilesFound");
S
Siying Dong 已提交
2116 2117 2118 2119 2120 2121 2122 2123 2124 2125 2126 2127
      // 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 已提交
2128
    TEST_SYNC_POINT("DBImpl::BackgroundCallFlush:ContextCleanedUp");
S
Siying Dong 已提交
2129 2130 2131 2132 2133 2134

    assert(num_running_flushes_ > 0);
    num_running_flushes_--;
    bg_flush_scheduled_--;
    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2135
    atomic_flush_install_cv_.SignalAll();
S
Siying Dong 已提交
2136 2137 2138 2139 2140 2141 2142 2143
    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.
  }
}

2144 2145
void DBImpl::BackgroundCallCompaction(PrepickedCompaction* prepicked_compaction,
                                      Env::Priority bg_thread_pri) {
S
Siying Dong 已提交
2146 2147 2148 2149 2150 2151 2152 2153 2154 2155 2156 2157 2158 2159 2160 2161 2162
  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_++;

    auto pending_outputs_inserted_elem =
        CaptureCurrentFileNumberInPendingOutputs();

2163 2164 2165 2166 2167
    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,
                                    prepicked_compaction);
S
Siying Dong 已提交
2168
    TEST_SYNC_POINT("BackgroundCallCompaction:1");
2169 2170 2171 2172 2173 2174
    if (s.IsBusy()) {
      bg_cv_.SignalAll();  // In case a waiter can proceed despite the error
      mutex_.Unlock();
      env_->SleepForMicroseconds(10000); // prevent hot loop
      mutex_.Lock();
    } else if (!s.ok() && !s.IsShutdownInProgress()) {
S
Siying Dong 已提交
2175 2176 2177 2178 2179 2180 2181 2182 2183 2184 2185 2186 2187 2188 2189 2190 2191 2192 2193 2194 2195 2196 2197 2198
      // 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();
    }

    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()
    FindObsoleteFiles(&job_context, !s.ok() && !s.IsShutdownInProgress());
2199
    TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:FoundObsoleteFiles");
S
Siying Dong 已提交
2200 2201

    // delete unnecessary files if any, this is done outside the mutex
2202 2203
    if (job_context.HaveSomethingToClean() ||
        job_context.HaveSomethingToDelete() || !log_buffer.IsEmpty()) {
S
Siying Dong 已提交
2204 2205 2206 2207 2208 2209 2210 2211 2212
      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);
2213
        TEST_SYNC_POINT("DBImpl::BackgroundCallCompaction:PurgedObsoleteFiles");
S
Siying Dong 已提交
2214 2215 2216 2217 2218 2219 2220
      }
      job_context.Clean();
      mutex_.Lock();
    }

    assert(num_running_compactions_ > 0);
    num_running_compactions_--;
2221 2222 2223 2224 2225 2226
    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 已提交
2227 2228 2229 2230 2231

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

    // See if there's more work to be done
    MaybeScheduleFlushOrCompaction();
2232 2233 2234
    if (made_progress ||
        (bg_compaction_scheduled_ == 0 &&
         bg_bottom_compaction_scheduled_ == 0) ||
2235
        HasPendingManualCompaction() || unscheduled_compactions_ == 0) {
S
Siying Dong 已提交
2236 2237
      // signal if
      // * made_progress -- need to wakeup DelayWrite
2238
      // * bg_{bottom,}_compaction_scheduled_ == 0 -- need to wakeup ~DBImpl
S
Siying Dong 已提交
2239 2240 2241 2242 2243 2244 2245 2246 2247 2248 2249 2250 2251 2252
      // * 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,
2253 2254 2255 2256 2257 2258
                                    LogBuffer* log_buffer,
                                    PrepickedCompaction* prepicked_compaction) {
  ManualCompactionState* manual_compaction =
      prepicked_compaction == nullptr
          ? nullptr
          : prepicked_compaction->manual_compaction_state;
S
Siying Dong 已提交
2259 2260 2261 2262 2263
  *made_progress = false;
  mutex_.AssertHeld();
  TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Start");

  bool is_manual = (manual_compaction != nullptr);
2264
  std::unique_ptr<Compaction> c;
2265 2266 2267 2268 2269
  if (prepicked_compaction != nullptr &&
      prepicked_compaction->compaction != nullptr) {
    c.reset(prepicked_compaction->compaction);
  }
  bool is_prepicked = is_manual || c;
S
Siying Dong 已提交
2270 2271 2272 2273 2274 2275

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

  CompactionJobStats compaction_job_stats;
2276 2277 2278 2279 2280 2281 2282
  Status status;
  if (!error_handler_.IsBGWorkStopped()) {
    if (shutting_down_.load(std::memory_order_acquire)) {
      status = Status::ShutdownInProgress();
    }
  } else {
    status = error_handler_.GetBGError();
2283 2284 2285 2286 2287
    // 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 已提交
2288 2289 2290 2291 2292 2293 2294 2295 2296
  }

  if (!status.ok()) {
    if (is_manual) {
      manual_compaction->status = status;
      manual_compaction->done = true;
      manual_compaction->in_progress = false;
      manual_compaction = nullptr;
    }
2297 2298 2299 2300
    if (c) {
      c->ReleaseCompactionFiles(status);
      c.reset();
    }
S
Siying Dong 已提交
2301 2302 2303 2304 2305 2306 2307 2308
    return status;
  }

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

2309 2310
  std::unique_ptr<TaskLimiterToken> task_token;

S
Siying Dong 已提交
2311 2312
  // InternalKey manual_end_storage;
  // InternalKey* manual_end = &manual_end_storage;
2313
  bool sfm_reserved_compact_space = false;
S
Siying Dong 已提交
2314
  if (is_manual) {
2315
    ManualCompactionState* m = manual_compaction;
S
Siying Dong 已提交
2316 2317 2318 2319 2320 2321 2322 2323 2324 2325 2326
    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,
                       (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
                       (m->end ? m->end->DebugString().c_str() : "(end)"));
    } else {
2327 2328
      // First check if we have enough room to do the compaction
      bool enough_room = EnoughRoomForCompaction(
2329
          m->cfd, *(c->inputs()), &sfm_reserved_compact_space, log_buffer);
2330 2331 2332 2333 2334 2335 2336 2337 2338 2339 2340 2341 2342 2343 2344 2345 2346 2347 2348 2349

      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(),
            (m->begin ? m->begin->DebugString().c_str() : "(begin)"),
            (m->end ? m->end->DebugString().c_str() : "(end)"),
            ((m->done || m->manual_end == nullptr)
                 ? "(end)"
                 : m->manual_end->DebugString().c_str()));
      }
S
Siying Dong 已提交
2350
    }
2351
  } else if (!is_prepicked && !compaction_queue_.empty()) {
2352
    if (HasExclusiveManualCompaction()) {
2353 2354 2355
      // Can't compact right now, but try again later
      TEST_SYNC_POINT("DBImpl::BackgroundCompaction()::Conflict");

Y
yizhu.sun 已提交
2356
      // Stay in the compaction queue.
2357 2358 2359 2360 2361
      unscheduled_compactions_++;

      return Status::OK();
    }

2362 2363 2364 2365 2366 2367 2368 2369
    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 已提交
2370 2371 2372 2373 2374 2375 2376 2377
    // We unreference here because the following code will take a Ref() on
    // this cfd if it is going to use it (Compaction class holds a
    // reference).
    // This will all happen under a mutex so we don't have to be afraid of
    // somebody else deleting it.
    if (cfd->Unref()) {
      // This was the last reference of the column family, so no need to
      // compact.
2378
      delete cfd;
S
Siying Dong 已提交
2379 2380 2381 2382 2383 2384 2385 2386 2387 2388 2389 2390 2391 2392 2393 2394
      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");
2395

S
Siying Dong 已提交
2396
      if (c != nullptr) {
2397
        bool enough_room = EnoughRoomForCompaction(
2398
            cfd, *(c->inputs()), &sfm_reserved_compact_space, log_buffer);
2399

2400 2401 2402 2403 2404 2405 2406 2407
        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 已提交
2408 2409
          AddToCompactionQueue(cfd);
          ++unscheduled_compactions_;
2410 2411 2412 2413 2414 2415 2416 2417 2418 2419 2420 2421 2422 2423 2424 2425 2426 2427 2428 2429 2430 2431 2432 2433 2434 2435 2436

          c.reset();
          // Don't need to sleep here, because BackgroundCallCompaction
          // will sleep if !s.ok()
          status = Status::CompactionTooLarge();
        } else {
          // update statistics
          MeasureTime(stats_, NUM_FILES_IN_SINGLE_COMPACTION,
                      c->inputs(0)->size());
          // There are three things that can change compaction score:
          // 1) When flush or compaction finish. This case is covered by
          // 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 已提交
2437 2438 2439 2440 2441 2442 2443 2444 2445 2446 2447
        }
      }
    }
  }

  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
2448 2449
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2450 2451 2452 2453 2454 2455 2456
    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 已提交
2457 2458 2459
    NotifyOnCompactionBegin(c->column_family_data(), c.get(), status,
                            compaction_job_stats, job_context->job_id);

S
Siying Dong 已提交
2460 2461 2462 2463 2464 2465
    for (const auto& f : *c->inputs(0)) {
      c->edit()->DeleteFile(c->level(), f->fd.GetNumber());
    }
    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
Y
Yanqin Jin 已提交
2466 2467 2468
    InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                       &job_context->superversion_contexts[0],
                                       *c->mutable_cf_options());
S
Siying Dong 已提交
2469 2470 2471 2472
    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;
2473 2474
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2475 2476
  } else if (!trivial_move_disallowed && c->IsTrivialMove()) {
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:TrivialMove");
2477 2478
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2479 2480 2481 2482 2483 2484 2485 2486 2487
    // 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 已提交
2488 2489 2490
    NotifyOnCompactionBegin(c->column_family_data(), c.get(), status,
                            compaction_job_stats, job_context->job_id);

S
Siying Dong 已提交
2491 2492 2493 2494 2495 2496 2497 2498 2499 2500 2501 2502
    // 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,
2503 2504
                           f->largest, f->fd.smallest_seqno,
                           f->fd.largest_seqno, f->marked_for_compaction);
S
Siying Dong 已提交
2505

2506 2507 2508 2509 2510
        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 已提交
2511 2512 2513 2514 2515 2516 2517 2518 2519
        ++moved_files;
        moved_bytes += f->fd.GetFileSize();
      }
    }

    status = versions_->LogAndApply(c->column_family_data(),
                                    *c->mutable_cf_options(), c->edit(),
                                    &mutex_, directories_.GetDbDir());
    // Use latest MutableCFOptions
Y
Yanqin Jin 已提交
2520 2521 2522
    InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                       &job_context->superversion_contexts[0],
                                       *c->mutable_cf_options());
S
Siying Dong 已提交
2523 2524 2525 2526 2527 2528 2529 2530 2531 2532 2533 2534 2535 2536 2537 2538 2539 2540 2541 2542 2543

    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();
2544 2545
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction",
                             c->column_family_data());
2546
  } else if (!is_prepicked && c->output_level() > 0 &&
2547 2548 2549 2550 2551 2552 2553
             c->output_level() ==
                 c->column_family_data()
                     ->current()
                     ->storage_info()
                     ->MaxOutputLevel(
                         immutable_db_options_.allow_ingest_behind) &&
             env_->GetBackgroundThreads(Env::Priority::BOTTOM) > 0) {
2554 2555 2556
    // 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.
2557 2558 2559 2560 2561 2562
    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;
2563 2564
    // Transfer requested token, so it doesn't need to do it again.
    ca->prepicked_compaction->task_token = std::move(task_token);
2565 2566 2567
    ++bg_bottom_compaction_scheduled_;
    env_->Schedule(&DBImpl::BGWorkBottomCompaction, ca, Env::Priority::BOTTOM,
                   this, &DBImpl::UnscheduleCallback);
S
Siying Dong 已提交
2568
  } else {
2569 2570
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:BeforeCompaction",
                             c->column_family_data());
2571
    int output_level __attribute__((__unused__));
2572
    output_level = c->output_level();
S
Siying Dong 已提交
2573 2574
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:NonTrivial",
                             &output_level);
2575
    std::vector<SequenceNumber> snapshot_seqs;
S
Siying Dong 已提交
2576
    SequenceNumber earliest_write_conflict_snapshot;
2577 2578 2579
    SnapshotChecker* snapshot_checker;
    GetSnapshotContext(job_context, &snapshot_seqs,
                       &earliest_write_conflict_snapshot, &snapshot_checker);
S
Siying Dong 已提交
2580 2581
    assert(is_snapshot_supported_ || snapshots_.empty());
    CompactionJob compaction_job(
2582 2583
        job_context->job_id, c.get(), immutable_db_options_,
        env_options_for_compaction_, versions_.get(), &shutting_down_,
2584
        preserve_deletes_seqnum_.load(), log_buffer, directories_.GetDbDir(),
2585
        GetDataDir(c->column_family_data(), c->output_path_id()), stats_,
2586
        &mutex_, &error_handler_, snapshot_seqs, earliest_write_conflict_snapshot,
2587
        snapshot_checker, table_cache_, &event_logger_,
S
Siying Dong 已提交
2588 2589 2590 2591 2592
        c->mutable_cf_options()->paranoid_file_checks,
        c->mutable_cf_options()->report_bg_io_stats, dbname_,
        &compaction_job_stats);
    compaction_job.Prepare();

P
Peter Pei 已提交
2593 2594 2595
    NotifyOnCompactionBegin(c->column_family_data(), c.get(), status,
                            compaction_job_stats, job_context->job_id);

S
Siying Dong 已提交
2596 2597 2598 2599 2600 2601 2602
    mutex_.Unlock();
    compaction_job.Run();
    TEST_SYNC_POINT("DBImpl::BackgroundCompaction:NonTrivial:AfterRun");
    mutex_.Lock();

    status = compaction_job.Install(*c->mutable_cf_options());
    if (status.ok()) {
Y
Yanqin Jin 已提交
2603 2604 2605
      InstallSuperVersionAndScheduleWork(c->column_family_data(),
                                         &job_context->superversion_contexts[0],
                                         *c->mutable_cf_options());
S
Siying Dong 已提交
2606 2607
    }
    *made_progress = true;
2608 2609
    TEST_SYNC_POINT_CALLBACK("DBImpl::BackgroundCompaction:AfterCompaction",
                             c->column_family_data());
S
Siying Dong 已提交
2610 2611 2612 2613
  }
  if (c != nullptr) {
    c->ReleaseCompactionFiles(status);
    *made_progress = true;
2614 2615 2616 2617 2618

#ifndef ROCKSDB_LITE
    // Need to make sure SstFileManager does its bookkeeping
    auto sfm = static_cast<SstFileManagerImpl*>(
        immutable_db_options_.sst_file_manager.get());
2619
    if (sfm && sfm_reserved_compact_space) {
2620 2621 2622 2623
      sfm->OnCompactionCompletion(c.get());
    }
#endif  // ROCKSDB_LITE

2624 2625
    NotifyOnCompactionCompleted(c->column_family_data(), c.get(), status,
                                compaction_job_stats, job_context->job_id);
S
Siying Dong 已提交
2626 2627
  }

2628
  if (status.ok() || status.IsCompactionTooLarge()) {
S
Siying Dong 已提交
2629 2630 2631 2632 2633 2634
    // Done
  } else if (status.IsShutdownInProgress()) {
    // Ignore compaction errors found during shutting down
  } else {
    ROCKS_LOG_WARN(immutable_db_options_.info_log, "Compaction error: %s",
                   status.ToString().c_str());
2635
    error_handler_.SetBGError(status, BackgroundErrorReason::kCompaction);
2636 2637 2638 2639 2640 2641 2642 2643 2644 2645 2646 2647 2648 2649 2650 2651 2652
    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 已提交
2653
  }
2654 2655
  // this will unref its input_version and column_family_data
  c.reset();
S
Siying Dong 已提交
2656 2657

  if (is_manual) {
2658
    ManualCompactionState* m = manual_compaction;
S
Siying Dong 已提交
2659 2660 2661 2662 2663 2664 2665 2666 2667 2668 2669 2670 2671 2672 2673 2674 2675 2676 2677 2678 2679 2680 2681 2682 2683 2684 2685 2686 2687 2688 2689 2690
    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;
    }
2691
    m->in_progress = false;  // not being processed anymore
S
Siying Dong 已提交
2692 2693 2694 2695 2696 2697 2698 2699 2700
  }
  TEST_SYNC_POINT("DBImpl::BackgroundCompaction:Finish");
  return status;
}

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

2701
void DBImpl::AddManualCompaction(DBImpl::ManualCompactionState* m) {
S
Siying Dong 已提交
2702 2703 2704
  manual_compaction_dequeue_.push_back(m);
}

2705
void DBImpl::RemoveManualCompaction(DBImpl::ManualCompactionState* m) {
S
Siying Dong 已提交
2706
  // Remove from queue
2707
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
2708 2709 2710 2711 2712 2713 2714 2715 2716 2717 2718 2719
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if (m == (*it)) {
      it = manual_compaction_dequeue_.erase(it);
      return;
    }
    it++;
  }
  assert(false);
  return;
}

2720
bool DBImpl::ShouldntRunManualCompaction(ManualCompactionState* m) {
S
Siying Dong 已提交
2721 2722 2723 2724 2725 2726
  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) {
2727 2728
    return (bg_bottom_compaction_scheduled_ > 0 ||
            bg_compaction_scheduled_ > 0);
S
Siying Dong 已提交
2729
  }
2730
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
2731 2732 2733 2734 2735 2736 2737 2738 2739 2740 2741 2742 2743 2744 2745 2746 2747 2748 2749 2750
      manual_compaction_dequeue_.begin();
  bool seen = false;
  while (it != manual_compaction_dequeue_.end()) {
    if (m == (*it)) {
      it++;
      seen = true;
      continue;
    } else if (MCOverlap(m, (*it)) && (!seen && !(*it)->in_progress)) {
      // Consider the other manual compaction *it, conflicts if:
      // overlaps with m
      // and (*it) is ahead in the queue and is not yet in progress
      return true;
    }
    it++;
  }
  return false;
}

bool DBImpl::HaveManualCompaction(ColumnFamilyData* cfd) {
  // Remove from priority queue
2751
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
2752 2753 2754 2755 2756 2757 2758
      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 已提交
2759
      // in progress
S
Siying Dong 已提交
2760 2761 2762 2763 2764 2765 2766 2767 2768
      return true;
    }
    it++;
  }
  return false;
}

bool DBImpl::HasExclusiveManualCompaction() {
  // Remove from priority queue
2769
  std::deque<ManualCompactionState*>::iterator it =
S
Siying Dong 已提交
2770 2771 2772 2773 2774 2775 2776 2777 2778 2779
      manual_compaction_dequeue_.begin();
  while (it != manual_compaction_dequeue_.end()) {
    if ((*it)->exclusive) {
      return true;
    }
    it++;
  }
  return false;
}

2780
bool DBImpl::MCOverlap(ManualCompactionState* m, ManualCompactionState* m1) {
S
Siying Dong 已提交
2781 2782 2783 2784 2785 2786 2787 2788 2789
  if ((m->exclusive) || (m1->exclusive)) {
    return true;
  }
  if (m->cfd != m1->cfd) {
    return false;
  }
  return true;
}

2790 2791 2792 2793 2794 2795 2796 2797 2798 2799 2800 2801 2802 2803 2804 2805 2806 2807 2808 2809 2810 2811 2812 2813 2814 2815 2816 2817 2818 2819 2820 2821 2822 2823 2824 2825 2826 2827 2828
#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)) {
      auto fn = TableFileName(c->immutable_cf_options()->cf_paths,
                              fmd->fd.GetNumber(), fmd->fd.GetPathId());
      compaction_job_info->input_files.push_back(fn);
      if (compaction_job_info->table_properties.count(fn) == 0) {
        shared_ptr<const TableProperties> tp;
        auto s = current->GetTableProperties(&tp, fmd, &fn);
        if (s.ok()) {
          compaction_job_info->table_properties[fn] = tp;
        }
      }
    }
  }
  for (const auto& newf : c->edit()->GetNewFiles()) {
    compaction_job_info->output_files.push_back(
        TableFileName(c->immutable_cf_options()->cf_paths,
                      newf.second.fd.GetNumber(), newf.second.fd.GetPathId()));
  }
}
#endif

2829
// SuperVersionContext gets created and destructed outside of the lock --
Y
Yanqin Jin 已提交
2830
// we use this conveniently to:
S
Siying Dong 已提交
2831 2832 2833 2834
// * 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
2835
// same sv_context, we can't reuse the SuperVersion() that got
S
Siying Dong 已提交
2836 2837 2838 2839 2840
// 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

2841 2842
void DBImpl::InstallSuperVersionAndScheduleWork(
    ColumnFamilyData* cfd, SuperVersionContext* sv_context,
Y
Yanqin Jin 已提交
2843
    const MutableCFOptions& mutable_cf_options) {
S
Siying Dong 已提交
2844 2845 2846 2847 2848 2849 2850 2851 2852 2853
  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;
  }

2854 2855
  // this branch is unlikely to step in
  if (UNLIKELY(sv_context->new_superversion == nullptr)) {
2856 2857
    sv_context->NewSuperVersion();
  }
2858
  cfd->InstallSuperVersion(sv_context, &mutex_, mutable_cf_options);
S
Siying Dong 已提交
2859 2860 2861 2862 2863 2864 2865

  // Whenever we install new SuperVersion, we might need to issue new flushes or
  // compactions.
  SchedulePendingCompaction(cfd);
  MaybeScheduleFlushOrCompaction();

  // Update max_total_in_memory_state_
2866 2867 2868
  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 已提交
2869
}
Y
Yi Wu 已提交
2870

2871 2872 2873 2874 2875 2876 2877 2878 2879 2880 2881 2882 2883 2884 2885 2886 2887 2888 2889 2890 2891 2892 2893 2894 2895 2896 2897 2898 2899 2900 2901
// ShouldPurge is called by FindObsoleteFiles when doing a full scan,
// and db mutex (mutex_) should already be held. This function performs a
// linear scan of an vector (files_grabbed_for_purge_) in search of a
// certain element. We expect FindObsoleteFiles with full scan to occur once
// every 10 hours by default, and the size of the vector is small.
// Therefore, the cost is affordable even if the mutex is held.
// 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.
// In the future, if we want to reduce the cost of search, we may try to keep
// the vector sorted.
bool DBImpl::ShouldPurge(uint64_t file_number) const {
  for (auto fn : files_grabbed_for_purge_) {
    if (file_number == fn) {
      return false;
    }
  }
  for (const auto& purge_file_info : purge_queue_) {
    if (purge_file_info.number == file_number) {
      return false;
    }
  }
  return true;
}

// MarkAsGrabbedForPurge is called by FindObsoleteFiles, and db mutex
// (mutex_) should already be held.
void DBImpl::MarkAsGrabbedForPurge(uint64_t file_number) {
  files_grabbed_for_purge_.emplace_back(file_number);
}

Y
Yi Wu 已提交
2902 2903 2904 2905 2906 2907 2908 2909
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);
}
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

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);
}
S
Siying Dong 已提交
2937
}  // namespace rocksdb