compaction_job.cc 72.7 KB
Newer Older
1
//  Copyright (c) 2011-present, Facebook, Inc.  All rights reserved.
S
Siying Dong 已提交
2 3 4
//  This source code is licensed under both the GPLv2 (found in the
//  COPYING file in the root directory) and Apache 2.0 License
//  (found in the LICENSE.Apache file in the root directory).
I
Igor Canadi 已提交
5 6 7 8 9
//
// 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.

10 11
#include "db/compaction/compaction_job.h"

I
Igor Canadi 已提交
12
#include <algorithm>
13
#include <cinttypes>
14
#include <functional>
I
Igor Canadi 已提交
15
#include <list>
16 17
#include <memory>
#include <random>
18 19 20
#include <set>
#include <thread>
#include <utility>
21
#include <vector>
I
Igor Canadi 已提交
22

23 24
#include "db/blob/blob_file_addition.h"
#include "db/blob/blob_file_builder.h"
I
Igor Canadi 已提交
25
#include "db/builder.h"
26
#include "db/db_impl/db_impl.h"
I
Igor Canadi 已提交
27 28
#include "db/db_iter.h"
#include "db/dbformat.h"
29
#include "db/error_handler.h"
30
#include "db/event_helpers.h"
I
Igor Canadi 已提交
31 32 33 34 35
#include "db/log_reader.h"
#include "db/log_writer.h"
#include "db/memtable.h"
#include "db/memtable_list.h"
#include "db/merge_context.h"
36
#include "db/merge_helper.h"
37
#include "db/output_validator.h"
38
#include "db/range_del_aggregator.h"
I
Igor Canadi 已提交
39
#include "db/version_set.h"
40
#include "file/filename.h"
41
#include "file/read_write_util.h"
42
#include "file/sst_file_manager_impl.h"
43
#include "file/writable_file_writer.h"
44 45
#include "logging/log_buffer.h"
#include "logging/logging.h"
46 47 48
#include "monitoring/iostats_context_imp.h"
#include "monitoring/perf_context_imp.h"
#include "monitoring/thread_status_util.h"
49
#include "port/port.h"
I
Igor Canadi 已提交
50 51
#include "rocksdb/db.h"
#include "rocksdb/env.h"
52
#include "rocksdb/sst_partitioner.h"
I
Igor Canadi 已提交
53 54 55
#include "rocksdb/statistics.h"
#include "rocksdb/status.h"
#include "rocksdb/table.h"
56 57
#include "table/block_based/block.h"
#include "table/block_based/block_based_table_factory.h"
58
#include "table/merging_iterator.h"
I
Igor Canadi 已提交
59
#include "table/table_builder.h"
60
#include "test_util/sync_point.h"
I
Igor Canadi 已提交
61
#include "util/coding.h"
62
#include "util/hash.h"
I
Igor Canadi 已提交
63
#include "util/mutexlock.h"
64
#include "util/random.h"
I
Igor Canadi 已提交
65
#include "util/stop_watch.h"
66
#include "util/string_util.h"
I
Igor Canadi 已提交
67

68
namespace ROCKSDB_NAMESPACE {
I
Igor Canadi 已提交
69

70 71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 96 97 98 99 100 101
const char* GetCompactionReasonString(CompactionReason compaction_reason) {
  switch (compaction_reason) {
    case CompactionReason::kUnknown:
      return "Unknown";
    case CompactionReason::kLevelL0FilesNum:
      return "LevelL0FilesNum";
    case CompactionReason::kLevelMaxLevelSize:
      return "LevelMaxLevelSize";
    case CompactionReason::kUniversalSizeAmplification:
      return "UniversalSizeAmplification";
    case CompactionReason::kUniversalSizeRatio:
      return "UniversalSizeRatio";
    case CompactionReason::kUniversalSortedRunNum:
      return "UniversalSortedRunNum";
    case CompactionReason::kFIFOMaxSize:
      return "FIFOMaxSize";
    case CompactionReason::kFIFOReduceNumFiles:
      return "FIFOReduceNumFiles";
    case CompactionReason::kFIFOTtl:
      return "FIFOTtl";
    case CompactionReason::kManualCompaction:
      return "ManualCompaction";
    case CompactionReason::kFilesMarkedForCompaction:
      return "FilesMarkedForCompaction";
    case CompactionReason::kBottommostFiles:
      return "BottommostFiles";
    case CompactionReason::kTtl:
      return "Ttl";
    case CompactionReason::kFlush:
      return "Flush";
    case CompactionReason::kExternalSstIngestion:
      return "ExternalSstIngestion";
S
Sagar Vemuri 已提交
102 103
    case CompactionReason::kPeriodicCompaction:
      return "PeriodicCompaction";
104 105 106 107 108 109 110 111
    case CompactionReason::kNumOfReasons:
      // fall through
    default:
      assert(false);
      return "Invalid";
  }
}

112
// Maintains state for each sub-compaction
113
struct CompactionJob::SubcompactionState {
114
  const Compaction* compaction;
115
  std::unique_ptr<CompactionIterator> c_iter;
I
Igor Canadi 已提交
116

117 118 119 120 121
  // The boundaries of the key-range this compaction is interested in. No two
  // subcompactions may have overlapping key-ranges.
  // 'start' is inclusive, 'end' is exclusive, and nullptr means unbounded
  Slice *start, *end;

122
  // The return status of this subcompaction
123 124
  Status status;

125 126 127
  // The return IO Status of this subcompaction
  IOStatus io_status;

128
  // Files produced by this subcompaction
I
Igor Canadi 已提交
129
  struct Output {
130 131 132 133 134
    Output(FileMetaData&& _meta, const InternalKeyComparator& _icmp,
           bool _enable_order_check, bool _enable_hash)
        : meta(std::move(_meta)),
          validator(_icmp, _enable_order_check, _enable_hash),
          finished(false) {}
135
    FileMetaData meta;
136
    OutputValidator validator;
137
    bool finished;
138
    std::shared_ptr<const TableProperties> table_properties;
I
Igor Canadi 已提交
139 140 141
  };

  // State kept for output being generated
142
  std::vector<Output> outputs;
143
  std::vector<BlobFileAddition> blob_file_additions;
144
  std::unique_ptr<WritableFileWriter> outfile;
I
Igor Canadi 已提交
145
  std::unique_ptr<TableBuilder> builder;
146

147
  Output* current_output() {
148
    if (outputs.empty()) {
149
      // This subcompaction's output could be empty if compaction was aborted
150 151
      // before this subcompaction had a chance to generate any output files.
      // When subcompactions are executed sequentially this is more likely and
152
      // will be particulalry likely for the later subcompactions to be empty.
153 154 155 156 157
      // Once they are run in parallel however it should be much rarer.
      return nullptr;
    } else {
      return &outputs.back();
    }
158
  }
I
Igor Canadi 已提交
159

160
  uint64_t current_output_file_size = 0;
161

162
  // State during the subcompaction
163 164
  uint64_t total_bytes = 0;
  uint64_t num_output_records = 0;
165
  CompactionJobStats compaction_job_stats;
166
  uint64_t approx_size = 0;
167
  // An index that used to speed up ShouldStopBefore().
168 169
  size_t grandparent_index = 0;
  // The number of bytes overlapping between the current output and
170
  // grandparent files used in ShouldStopBefore().
171
  uint64_t overlapped_bytes = 0;
172
  // A flag determine whether the key has been seen in ShouldStopBefore()
173
  bool seen_key = false;
174

175 176
  SubcompactionState(Compaction* c, Slice* _start, Slice* _end, uint64_t size)
      : compaction(c), start(_start), end(_end), approx_size(size) {
D
Dmitri Smirnov 已提交
177 178
    assert(compaction != nullptr);
  }
D
Dmitri Smirnov 已提交
179

180 181
  // Adds the key and value to the builder
  // If paranoid is true, adds the key-value to the paranoid hash
182
  Status AddToBuilder(const Slice& key, const Slice& value) {
183 184 185
    auto curr = current_output();
    assert(builder != nullptr);
    assert(curr != nullptr);
186 187 188
    Status s = curr->validator.Add(key, value);
    if (!s.ok()) {
      return s;
189 190
    }
    builder->Add(key, value);
191
    return Status::OK();
192 193
  }

194 195
  // Returns true iff we should stop building the current output
  // before processing "internal_key".
196
  bool ShouldStopBefore(const Slice& internal_key, uint64_t curr_file_size) {
197 198 199 200 201 202 203 204 205 206 207 208 209 210 211
    const InternalKeyComparator* icmp =
        &compaction->column_family_data()->internal_comparator();
    const std::vector<FileMetaData*>& grandparents = compaction->grandparents();

    // Scan to find earliest grandparent file that contains key.
    while (grandparent_index < grandparents.size() &&
           icmp->Compare(internal_key,
                         grandparents[grandparent_index]->largest.Encode()) >
               0) {
      if (seen_key) {
        overlapped_bytes += grandparents[grandparent_index]->fd.GetFileSize();
      }
      assert(grandparent_index + 1 >= grandparents.size() ||
             icmp->Compare(
                 grandparents[grandparent_index]->largest.Encode(),
212
                 grandparents[grandparent_index + 1]->smallest.Encode()) <= 0);
213 214 215 216
      grandparent_index++;
    }
    seen_key = true;

217 218
    if (overlapped_bytes + curr_file_size >
        compaction->max_compaction_bytes()) {
219 220 221 222 223 224 225
      // Too much overlap for current output; start new output
      overlapped_bytes = 0;
      return true;
    }

    return false;
  }
226 227 228 229 230
};

// Maintains state for the entire compaction
struct CompactionJob::CompactionState {
  Compaction* const compaction;
I
Igor Canadi 已提交
231

232 233
  // REQUIRED: subcompaction states are stored in order of increasing
  // key-range
234
  std::vector<CompactionJob::SubcompactionState> sub_compact_states;
235 236
  Status status;

237 238 239 240 241
  size_t num_output_files = 0;
  uint64_t total_bytes = 0;
  size_t num_blob_output_files = 0;
  uint64_t total_blob_bytes = 0;
  uint64_t num_output_records = 0;
I
Igor Canadi 已提交
242

243
  explicit CompactionState(Compaction* c) : compaction(c) {}
244 245

  Slice SmallestUserKey() {
246 247 248 249
    for (const auto& sub_compact_state : sub_compact_states) {
      if (!sub_compact_state.outputs.empty() &&
          sub_compact_state.outputs[0].finished) {
        return sub_compact_state.outputs[0].meta.smallest.user_key();
250 251
      }
    }
252
    // If there is no finished output, return an empty slice.
253
    return Slice(nullptr, 0);
254 255 256
  }

  Slice LargestUserKey() {
257 258 259 260 261
    for (auto it = sub_compact_states.rbegin(); it < sub_compact_states.rend();
         ++it) {
      if (!it->outputs.empty() && it->current_output()->finished) {
        assert(it->current_output() != nullptr);
        return it->current_output()->meta.largest.user_key();
262 263
      }
    }
264
    // If there is no finished output, return an empty slice.
265
    return Slice(nullptr, 0);
266
  }
I
Igor Canadi 已提交
267 268
};

269
void CompactionJob::AggregateStatistics() {
270 271
  assert(compact_);

272
  for (SubcompactionState& sc : compact_->sub_compact_states) {
273 274 275 276 277 278 279 280
    auto& outputs = sc.outputs;

    if (!outputs.empty() && !outputs.back().meta.fd.file_size) {
      // An error occurred, so ignore the last output.
      outputs.pop_back();
    }

    compact_->num_output_files += outputs.size();
281
    compact_->total_bytes += sc.total_bytes;
282 283 284 285 286 287 288 289 290

    const auto& blobs = sc.blob_file_additions;

    compact_->num_blob_output_files += blobs.size();

    for (const auto& blob : blobs) {
      compact_->total_blob_bytes += blob.GetTotalBlobBytes();
    }

291
    compact_->num_output_records += sc.num_output_records;
292

293
    compaction_job_stats_->Add(sc.compaction_job_stats);
294 295 296
  }
}

I
Igor Canadi 已提交
297
CompactionJob::CompactionJob(
298
    int job_id, Compaction* compaction, const ImmutableDBOptions& db_options,
299
    const FileOptions& file_options, VersionSet* versions,
300
    const std::atomic<bool>* shutting_down,
301
    const SequenceNumber preserve_deletes_seqnum, LogBuffer* log_buffer,
302 303
    FSDirectory* db_directory, FSDirectory* output_directory,
    FSDirectory* blob_output_directory, Statistics* stats,
304
    InstrumentedMutex* db_mutex, ErrorHandler* db_error_handler,
I
Igor Canadi 已提交
305
    std::vector<SequenceNumber> existing_snapshots,
306
    SequenceNumber earliest_write_conflict_snapshot,
Y
Yi Wu 已提交
307 308
    const SnapshotChecker* snapshot_checker, std::shared_ptr<Cache> table_cache,
    EventLogger* event_logger, bool paranoid_file_checks, bool measure_io_stats,
309
    const std::string& dbname, CompactionJobStats* compaction_job_stats,
310
    Env::Priority thread_pri, const std::shared_ptr<IOTracer>& io_tracer,
311
    const std::atomic<int>* manual_compaction_paused, const std::string& db_id,
312
    const std::string& db_session_id)
313 314
    : job_id_(job_id),
      compact_(new CompactionState(compaction)),
315
      compaction_job_stats_(compaction_job_stats),
316
      compaction_stats_(compaction->compaction_reason(), 1),
317
      dbname_(dbname),
318 319
      db_id_(db_id),
      db_session_id_(db_session_id),
I
Igor Canadi 已提交
320
      db_options_(db_options),
321
      file_options_(file_options),
I
Igor Canadi 已提交
322
      env_(db_options.env),
323
      io_tracer_(io_tracer),
324
      fs_(db_options.fs, io_tracer),
325 326
      file_options_for_read_(
          fs_->OptimizeForCompactionTableRead(file_options, db_options_)),
I
Igor Canadi 已提交
327 328
      versions_(versions),
      shutting_down_(shutting_down),
329
      manual_compaction_paused_(manual_compaction_paused),
330
      preserve_deletes_seqnum_(preserve_deletes_seqnum),
I
Igor Canadi 已提交
331 332
      log_buffer_(log_buffer),
      db_directory_(db_directory),
333
      output_directory_(output_directory),
334
      blob_output_directory_(blob_output_directory),
I
Igor Canadi 已提交
335
      stats_(stats),
336
      db_mutex_(db_mutex),
337
      db_error_handler_(db_error_handler),
I
Igor Canadi 已提交
338
      existing_snapshots_(std::move(existing_snapshots)),
339
      earliest_write_conflict_snapshot_(earliest_write_conflict_snapshot),
Y
Yi Wu 已提交
340
      snapshot_checker_(snapshot_checker),
I
Igor Canadi 已提交
341
      table_cache_(std::move(table_cache)),
I
Igor Canadi 已提交
342
      event_logger_(event_logger),
343
      bottommost_level_(false),
344
      paranoid_file_checks_(paranoid_file_checks),
S
Stream  
Shaohua Li 已提交
345
      measure_io_stats_(measure_io_stats),
346 347
      write_hint_(Env::WLTH_NOT_SET),
      thread_pri_(thread_pri) {
348
  assert(compaction_job_stats_ != nullptr);
349
  assert(log_buffer_ != nullptr);
350 351
  const auto* cfd = compact_->compaction->column_family_data();
  ThreadStatusUtil::SetColumnFamily(cfd, cfd->ioptions()->env,
Y
Yi Wu 已提交
352
                                    db_options_.enable_thread_tracking);
353
  ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);
354
  ReportStartedCompaction(compaction);
355 356 357 358 359 360
}

CompactionJob::~CompactionJob() {
  assert(compact_ == nullptr);
  ThreadStatusUtil::ResetThreadStatus();
}
I
Igor Canadi 已提交
361

362
void CompactionJob::ReportStartedCompaction(Compaction* compaction) {
363 364
  const auto* cfd = compact_->compaction->column_family_data();
  ThreadStatusUtil::SetColumnFamily(cfd, cfd->ioptions()->env,
Y
Yi Wu 已提交
365
                                    db_options_.enable_thread_tracking);
366

367 368
  ThreadStatusUtil::SetThreadOperationProperty(ThreadStatus::COMPACTION_JOB_ID,
                                               job_id_);
369 370 371 372 373 374

  ThreadStatusUtil::SetThreadOperationProperty(
      ThreadStatus::COMPACTION_INPUT_OUTPUT_LEVEL,
      (static_cast<uint64_t>(compact_->compaction->start_level()) << 32) +
          compact_->compaction->output_level());

375 376 377
  // In the current design, a CompactionJob is always created
  // for non-trivial compaction.
  assert(compaction->IsTrivialMove() == false ||
378
         compaction->is_manual_compaction() == true);
379

380 381
  ThreadStatusUtil::SetThreadOperationProperty(
      ThreadStatus::COMPACTION_PROP_FLAGS,
S
sdong 已提交
382
      compaction->is_manual_compaction() +
383
          (compaction->deletion_compaction() << 1));
384 385 386 387 388 389 390 391 392 393 394 395 396 397

  ThreadStatusUtil::SetThreadOperationProperty(
      ThreadStatus::COMPACTION_TOTAL_INPUT_BYTES,
      compaction->CalculateTotalInputSize());

  IOSTATS_RESET(bytes_written);
  IOSTATS_RESET(bytes_read);
  ThreadStatusUtil::SetThreadOperationProperty(
      ThreadStatus::COMPACTION_BYTES_WRITTEN, 0);
  ThreadStatusUtil::SetThreadOperationProperty(
      ThreadStatus::COMPACTION_BYTES_READ, 0);

  // Set the thread operation after operation properties
  // to ensure GetThreadList() can always show them all together.
398
  ThreadStatusUtil::SetThreadOperation(ThreadStatus::OP_COMPACTION);
399

400 401 402
  compaction_job_stats_->is_manual_compaction =
      compaction->is_manual_compaction();
  compaction_job_stats_->is_full_compaction = compaction->is_full_compaction();
403 404
}

I
Igor Canadi 已提交
405
void CompactionJob::Prepare() {
406 407
  AutoThreadOperationStageUpdater stage_updater(
      ThreadStatus::STAGE_COMPACTION_PREPARE);
I
Igor Canadi 已提交
408 409

  // Generate file_levels_ for compaction berfore making Iterator
410 411
  auto* c = compact_->compaction;
  assert(c->column_family_data() != nullptr);
412 413
  assert(c->column_family_data()->current()->storage_info()->NumLevelFiles(
             compact_->compaction->level()) > 0);
I
Igor Canadi 已提交
414

415 416
  write_hint_ =
      c->column_family_data()->CalculateSSTWriteHint(c->output_level());
417
  bottommost_level_ = c->bottommost_level();
418

419
  if (c->ShouldFormSubcompactions()) {
S
Siying Dong 已提交
420 421 422 423
    {
      StopWatch sw(env_, stats_, SUBCOMPACTION_SETUP_TIME);
      GenSubcompactionBoundaries();
    }
424 425 426 427 428 429 430
    assert(sizes_.size() == boundaries_.size() + 1);

    for (size_t i = 0; i <= boundaries_.size(); i++) {
      Slice* start = i == 0 ? nullptr : &boundaries_[i - 1];
      Slice* end = i == boundaries_.size() ? nullptr : &boundaries_[i];
      compact_->sub_compact_states.emplace_back(c, start, end, sizes_[i]);
    }
S
Siying Dong 已提交
431 432
    RecordInHistogram(stats_, NUM_SUBCOMPACTIONS_SCHEDULED,
                      compact_->sub_compact_states.size());
433
  } else {
434 435 436 437 438
    constexpr Slice* start = nullptr;
    constexpr Slice* end = nullptr;
    constexpr uint64_t size = 0;

    compact_->sub_compact_states.emplace_back(c, start, end, size);
439
  }
440 441
}

442 443 444
struct RangeWithSize {
  Range range;
  uint64_t size;
445

446 447 448 449 450 451 452
  RangeWithSize(const Slice& a, const Slice& b, uint64_t s = 0)
      : range(a, b), size(s) {}
};

void CompactionJob::GenSubcompactionBoundaries() {
  auto* c = compact_->compaction;
  auto* cfd = c->column_family_data();
453 454
  const Comparator* cfd_comparator = cfd->user_comparator();
  std::vector<Slice> bounds;
455 456 457 458
  int start_lvl = c->start_level();
  int out_lvl = c->output_level();

  // Add the starting and/or ending key of certain input files as a potential
459
  // boundary
460 461 462 463 464 465 466
  for (size_t lvl_idx = 0; lvl_idx < c->num_input_levels(); lvl_idx++) {
    int lvl = c->level(lvl_idx);
    if (lvl >= start_lvl && lvl <= out_lvl) {
      const LevelFilesBrief* flevel = c->input_levels(lvl_idx);
      size_t num_files = flevel->num_files;

      if (num_files == 0) {
467
        continue;
468
      }
A
Ari Ekmekji 已提交
469

470 471 472 473
      if (lvl == 0) {
        // For level 0 add the starting and ending key of each file since the
        // files may have greatly differing key ranges (not range-partitioned)
        for (size_t i = 0; i < num_files; i++) {
474 475
          bounds.emplace_back(flevel->files[i].smallest_key);
          bounds.emplace_back(flevel->files[i].largest_key);
476 477 478 479
        }
      } else {
        // For all other levels add the smallest/largest key in the level to
        // encompass the range covered by that level
480 481
        bounds.emplace_back(flevel->files[0].smallest_key);
        bounds.emplace_back(flevel->files[num_files - 1].largest_key);
482 483 484 485 486 487
        if (lvl == out_lvl) {
          // For the last level include the starting keys of all files since
          // the last level is the largest and probably has the widest key
          // range. Since it's range partitioned, the ending key of one file
          // and the starting key of the next are very close (or identical).
          for (size_t i = 1; i < num_files; i++) {
488
            bounds.emplace_back(flevel->files[i].smallest_key);
A
Ari Ekmekji 已提交
489
          }
490 491 492 493
        }
      }
    }
  }
494

495
  std::sort(bounds.begin(), bounds.end(),
496 497 498 499
            [cfd_comparator](const Slice& a, const Slice& b) -> bool {
              return cfd_comparator->Compare(ExtractUserKey(a),
                                             ExtractUserKey(b)) < 0;
            });
500
  // Remove duplicated entries from bounds
501 502 503 504 505 506 507
  bounds.erase(
      std::unique(bounds.begin(), bounds.end(),
                  [cfd_comparator](const Slice& a, const Slice& b) -> bool {
                    return cfd_comparator->Compare(ExtractUserKey(a),
                                                   ExtractUserKey(b)) == 0;
                  }),
      bounds.end());
508

509 510 511 512
  // Combine consecutive pairs of boundaries into ranges with an approximate
  // size of data covered by keys in that range
  uint64_t sum = 0;
  std::vector<RangeWithSize> ranges;
513 514 515 516
  // Get input version from CompactionState since it's already referenced
  // earlier in SetInputVersioCompaction::SetInputVersion and will not change
  // when db_mutex_ is released below
  auto* v = compact_->compaction->input_version();
517 518
  for (auto it = bounds.begin();;) {
    const Slice a = *it;
519
    ++it;
520 521 522 523 524 525

    if (it == bounds.end()) {
      break;
    }

    const Slice b = *it;
526 527 528 529 530

    // ApproximateSize could potentially create table reader iterator to seek
    // to the index block and may incur I/O cost in the process. Unlock db
    // mutex to reduce contention
    db_mutex_->Unlock();
531 532
    uint64_t size = versions_->ApproximateSize(SizeApproximationOptions(), v, a,
                                               b, start_lvl, out_lvl + 1,
533
                                               TableReaderCaller::kCompaction);
534
    db_mutex_->Lock();
535 536 537 538 539 540
    ranges.emplace_back(a, b, size);
    sum += size;
  }

  // Group the ranges into subcompactions
  const double min_file_fill_percent = 4.0 / 5;
541 542 543 544 545 546
  int base_level = v->storage_info()->base_level();
  uint64_t max_output_files = static_cast<uint64_t>(std::ceil(
      sum / min_file_fill_percent /
      MaxFileSizeForLevel(*(c->mutable_cf_options()), out_lvl,
          c->immutable_cf_options()->compaction_style, base_level,
          c->immutable_cf_options()->level_compaction_dynamic_level_bytes)));
547 548
  uint64_t subcompactions =
      std::min({static_cast<uint64_t>(ranges.size()),
549
                static_cast<uint64_t>(c->max_subcompactions()),
550 551 552
                max_output_files});

  if (subcompactions > 1) {
553
    double mean = sum * 1.0 / subcompactions;
554 555 556
    // Greedily add ranges to the subcompaction until the sum of the ranges'
    // sizes becomes >= the expected mean size of a subcompaction
    sum = 0;
557
    for (size_t i = 0; i + 1 < ranges.size(); i++) {
558
      sum += ranges[i].size;
559 560 561
      if (subcompactions == 1) {
        // If there's only one left to schedule then it goes to the end so no
        // need to put an end boundary
562
        continue;
563 564 565 566 567 568 569 570 571 572 573 574
      }
      if (sum >= mean) {
        boundaries_.emplace_back(ExtractUserKey(ranges[i].range.limit));
        sizes_.emplace_back(sum);
        subcompactions--;
        sum = 0;
      }
    }
    sizes_.emplace_back(sum + ranges.back().size);
  } else {
    // Only one range so its size is the total sum of sizes computed above
    sizes_.emplace_back(sum);
575
  }
I
Igor Canadi 已提交
576 577 578
}

Status CompactionJob::Run() {
579 580
  AutoThreadOperationStageUpdater stage_updater(
      ThreadStatus::STAGE_COMPACTION_RUN);
581
  TEST_SYNC_POINT("CompactionJob::Run():Start");
I
Igor Canadi 已提交
582
  log_buffer_->FlushBufferToLog();
583
  LogCompaction();
584

585 586
  const size_t num_threads = compact_->sub_compact_states.size();
  assert(num_threads > 0);
587
  const uint64_t start_micros = env_->NowMicros();
588 589

  // Launch a thread for each of subcompactions 1...num_threads-1
D
Dmitri Smirnov 已提交
590
  std::vector<port::Thread> thread_pool;
591 592 593 594
  thread_pool.reserve(num_threads - 1);
  for (size_t i = 1; i < compact_->sub_compact_states.size(); i++) {
    thread_pool.emplace_back(&CompactionJob::ProcessKeyValueCompaction, this,
                             &compact_->sub_compact_states[i]);
595
  }
596 597 598 599 600 601 602 603 604 605

  // Always schedule the first subcompaction (whether or not there are also
  // others) in the current thread to be efficient with resources
  ProcessKeyValueCompaction(&compact_->sub_compact_states[0]);

  // Wait for all other threads (if there are any) to finish execution
  for (auto& thread : thread_pool) {
    thread.join();
  }

606
  compaction_stats_.micros = env_->NowMicros() - start_micros;
607 608 609 610 611 612
  compaction_stats_.cpu_micros = 0;
  for (size_t i = 0; i < compact_->sub_compact_states.size(); i++) {
    compaction_stats_.cpu_micros +=
        compact_->sub_compact_states[i].compaction_job_stats.cpu_micros;
  }

S
Siying Dong 已提交
613 614 615
  RecordTimeToHistogram(stats_, COMPACTION_TIME, compaction_stats_.micros);
  RecordTimeToHistogram(stats_, COMPACTION_CPU_TIME,
                        compaction_stats_.cpu_micros);
616

617 618
  TEST_SYNC_POINT("CompactionJob::Run:BeforeVerify");

619
  // Check if any thread encountered an error during execution
620
  Status status;
621
  IOStatus io_s;
622 623
  bool wrote_new_blob_files = false;

624 625 626
  for (const auto& state : compact_->sub_compact_states) {
    if (!state.status.ok()) {
      status = state.status;
627
      io_s = state.io_status;
628 629
      break;
    }
630 631 632 633

    if (!state.blob_file_additions.empty()) {
      wrote_new_blob_files = true;
    }
634
  }
635

636 637 638
  if (io_status_.ok()) {
    io_status_ = io_s;
  }
639 640 641 642 643 644 645 646 647 648 649
  if (status.ok()) {
    constexpr IODebugContext* dbg = nullptr;

    if (output_directory_) {
      io_s = output_directory_->Fsync(IOOptions(), dbg);
    }

    if (io_s.ok() && wrote_new_blob_files && blob_output_directory_ &&
        blob_output_directory_ != output_directory_) {
      io_s = blob_output_directory_->Fsync(IOOptions(), dbg);
    }
650
  }
651
  if (io_status_.ok()) {
652
    io_status_ = io_s;
653 654
  }
  if (status.ok()) {
655
    status = io_s;
656
  }
657 658
  if (status.ok()) {
    thread_pool.clear();
659
    std::vector<const CompactionJob::SubcompactionState::Output*> files_output;
660 661
    for (const auto& state : compact_->sub_compact_states) {
      for (const auto& output : state.outputs) {
662
        files_output.emplace_back(&output);
663 664 665 666 667
      }
    }
    ColumnFamilyData* cfd = compact_->compaction->column_family_data();
    auto prefix_extractor =
        compact_->compaction->mutable_cf_options()->prefix_extractor.get();
668
    std::atomic<size_t> next_file_idx(0);
669 670
    auto verify_table = [&](Status& output_status) {
      while (true) {
671 672
        size_t file_idx = next_file_idx.fetch_add(1);
        if (file_idx >= files_output.size()) {
673 674 675 676 677 678 679 680
          break;
        }
        // Verify that the table is usable
        // We set for_compaction to false and don't OptimizeForCompactionTableRead
        // here because this is a special case after we finish the table building
        // No matter whether use_direct_io_for_flush_and_compaction is true,
        // we will regard this verification as user reads since the goal is
        // to cache it here for further user reads
681
        ReadOptions read_options;
682
        InternalIterator* iter = cfd->table_cache()->NewIterator(
683
            read_options, file_options_, cfd->internal_comparator(),
684 685
            files_output[file_idx]->meta, /*range_del_agg=*/nullptr,
            prefix_extractor,
686
            /*table_reader_ptr=*/nullptr,
687 688
            cfd->internal_stats()->GetFileReadHist(
                compact_->compaction->output_level()),
689 690
            TableReaderCaller::kCompactionRefill, /*arena=*/nullptr,
            /*skip_filters=*/false, compact_->compaction->output_level(),
691 692
            MaxFileSizeForL0MetaPin(
                *compact_->compaction->mutable_cf_options()),
693
            /*smallest_compaction_key=*/nullptr,
694 695
            /*largest_compaction_key=*/nullptr,
            /*allow_unprepared_value=*/false);
696 697 698
        auto s = iter->status();

        if (s.ok() && paranoid_file_checks_) {
699 700 701
          OutputValidator validator(cfd->internal_comparator(),
                                    /*_enable_order_check=*/true,
                                    /*_enable_hash=*/true);
702
          for (iter->SeekToFirst(); iter->Valid(); iter->Next()) {
703 704 705 706
            s = validator.Add(iter->key(), iter->value());
            if (!s.ok()) {
              break;
            }
707
          }
708 709 710 711 712
          if (s.ok()) {
            s = iter->status();
          }
          if (s.ok() &&
              !validator.CompareValidator(files_output[file_idx]->validator)) {
713
            s = Status::Corruption("Paranoid checksums do not match");
714
          }
715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740
        }

        delete iter;

        if (!s.ok()) {
          output_status = s;
          break;
        }
      }
    };
    for (size_t i = 1; i < compact_->sub_compact_states.size(); i++) {
      thread_pool.emplace_back(verify_table,
                               std::ref(compact_->sub_compact_states[i].status));
    }
    verify_table(compact_->sub_compact_states[0].status);
    for (auto& thread : thread_pool) {
      thread.join();
    }
    for (const auto& state : compact_->sub_compact_states) {
      if (!state.status.ok()) {
        status = state.status;
        break;
      }
    }
  }

741 742 743
  TablePropertiesCollection tp;
  for (const auto& state : compact_->sub_compact_states) {
    for (const auto& output : state.outputs) {
744 745 746
      auto fn =
          TableFileName(state.compaction->immutable_cf_options()->cf_paths,
                        output.meta.fd.GetNumber(), output.meta.fd.GetPathId());
747 748 749 750 751
      tp[fn] = output.table_properties;
    }
  }
  compact_->compaction->SetOutputTableProperties(std::move(tp));

752 753
  // Finish up all book-keeping to unify the subcompaction results
  AggregateStatistics();
754
  UpdateCompactionStats();
755

756 757 758 759
  RecordCompactionIOStats();
  LogFlush(db_options_.info_log);
  TEST_SYNC_POINT("CompactionJob::Run():End");

760
  compact_->status = status;
761 762 763
  return status;
}

764
Status CompactionJob::Install(const MutableCFOptions& mutable_cf_options) {
765 766
  assert(compact_);

767 768
  AutoThreadOperationStageUpdater stage_updater(
      ThreadStatus::STAGE_COMPACTION_INSTALL);
769
  db_mutex_->AssertHeld();
770
  Status status = compact_->status;
771

I
Igor Canadi 已提交
772
  ColumnFamilyData* cfd = compact_->compaction->column_family_data();
773 774
  assert(cfd);

I
Igor Canadi 已提交
775
  cfd->internal_stats()->AddCompactionStats(
776
      compact_->compaction->output_level(), thread_pri_, compaction_stats_);
I
Igor Canadi 已提交
777

778
  if (status.ok()) {
779
    status = InstallCompactionResults(mutable_cf_options);
I
Igor Canadi 已提交
780
  }
781 782 783
  if (!versions_->io_status().ok()) {
    io_status_ = versions_->io_status();
  }
784

I
Igor Canadi 已提交
785
  VersionStorageInfo::LevelSummaryStorage tmp;
786
  auto vstorage = cfd->current()->storage_info();
I
Igor Canadi 已提交
787
  const auto& stats = compaction_stats_;
788 789 790

  double read_write_amp = 0.0;
  double write_amp = 0.0;
791 792 793
  double bytes_read_per_sec = 0;
  double bytes_written_per_sec = 0;

794 795 796 797 798 799 800
  if (stats.bytes_read_non_output_levels > 0) {
    read_write_amp = (stats.bytes_written + stats.bytes_read_output_level +
                      stats.bytes_read_non_output_levels) /
                     static_cast<double>(stats.bytes_read_non_output_levels);
    write_amp = stats.bytes_written /
                static_cast<double>(stats.bytes_read_non_output_levels);
  }
801 802 803 804 805 806 807 808
  if (stats.micros > 0) {
    bytes_read_per_sec =
        (stats.bytes_read_non_output_levels + stats.bytes_read_output_level) /
        static_cast<double>(stats.micros);
    bytes_written_per_sec =
        stats.bytes_written / static_cast<double>(stats.micros);
  }

809 810
  const std::string& column_family_name = cfd->GetName();

811
  ROCKS_LOG_BUFFER(
812 813 814 815
      log_buffer_,
      "[%s] compacted to: %s, MB/sec: %.1f rd, %.1f wr, level %d, "
      "files in(%d, %d) out(%d) "
      "MB in(%.1f, %.1f) out(%.1f), read-write-amplify(%.1f) "
S
Siying Dong 已提交
816 817
      "write-amplify(%.1f) %s, records in: %" PRIu64
      ", records dropped: %" PRIu64 " output_compression: %s\n",
818 819 820
      column_family_name.c_str(), vstorage->LevelSummary(&tmp),
      bytes_read_per_sec, bytes_written_per_sec,
      compact_->compaction->output_level(),
821
      stats.num_input_files_in_non_output_levels,
822
      stats.num_input_files_in_output_level, stats.num_output_files,
823 824
      stats.bytes_read_non_output_levels / 1048576.0,
      stats.bytes_read_output_level / 1048576.0,
825
      stats.bytes_written / 1048576.0, read_write_amp, write_amp,
826
      status.ToString().c_str(), stats.num_input_records,
827 828 829
      stats.num_dropped_records,
      CompressionTypeToString(compact_->compaction->output_compression())
          .c_str());
I
Igor Canadi 已提交
830

831 832 833 834 835 836 837 838 839
  const auto& blob_files = vstorage->GetBlobFiles();
  if (!blob_files.empty()) {
    ROCKS_LOG_BUFFER(log_buffer_,
                     "[%s] Blob file summary: head=%" PRIu64 ", tail=%" PRIu64
                     "\n",
                     column_family_name.c_str(), blob_files.begin()->first,
                     blob_files.rbegin()->first);
  }

840 841
  UpdateCompactionJobStats(stats);

842
  auto stream = event_logger_->LogToBuffer(log_buffer_);
843 844
  stream << "job" << job_id_ << "event"
         << "compaction_finished"
845 846 847
         << "compaction_time_micros" << stats.micros
         << "compaction_time_cpu_micros" << stats.cpu_micros << "output_level"
         << compact_->compaction->output_level() << "num_output_files"
848 849 850 851 852 853 854 855 856 857 858 859
         << compact_->num_output_files << "total_output_size"
         << compact_->total_bytes;

  if (compact_->num_blob_output_files > 0) {
    stream << "num_blob_output_files" << compact_->num_blob_output_files
           << "total_blob_output_size" << compact_->total_blob_bytes;
  }

  stream << "num_input_records" << stats.num_input_records
         << "num_output_records" << compact_->num_output_records
         << "num_subcompactions" << compact_->sub_compact_states.size()
         << "output_compression"
860
         << CompressionTypeToString(compact_->compaction->output_compression());
861

862 863 864 865
  stream << "num_single_delete_mismatches"
         << compaction_job_stats_->num_single_del_mismatch;
  stream << "num_single_delete_fallthrough"
         << compaction_job_stats_->num_single_del_fallthru;
866

867
  if (measure_io_stats_) {
868 869 870 871 872 873 874 875
    stream << "file_write_nanos" << compaction_job_stats_->file_write_nanos;
    stream << "file_range_sync_nanos"
           << compaction_job_stats_->file_range_sync_nanos;
    stream << "file_fsync_nanos" << compaction_job_stats_->file_fsync_nanos;
    stream << "file_prepare_write_nanos"
           << compaction_job_stats_->file_prepare_write_nanos;
  }

876 877 878 879 880 881 882
  stream << "lsm_state";
  stream.StartArray();
  for (int level = 0; level < vstorage->num_levels(); ++level) {
    stream << vstorage->NumLevelFiles(level);
  }
  stream.EndArray();

883 884 885 886 887
  if (!blob_files.empty()) {
    stream << "blob_file_head" << blob_files.begin()->first;
    stream << "blob_file_tail" << blob_files.rbegin()->first;
  }

888 889
  CleanupCompaction();
  return status;
I
Igor Canadi 已提交
890 891
}

892
void CompactionJob::ProcessKeyValueCompaction(SubcompactionState* sub_compact) {
893 894
  assert(sub_compact);
  assert(sub_compact->compaction);
895 896 897

  uint64_t prev_cpu_micros = env_->NowCPUNanos() / 1000;

898
  ColumnFamilyData* cfd = sub_compact->compaction->column_family_data();
899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916

  // Create compaction filter and fail the compaction if
  // IgnoreSnapshots() = false because it is not supported anymore
  const CompactionFilter* compaction_filter =
      cfd->ioptions()->compaction_filter;
  std::unique_ptr<CompactionFilter> compaction_filter_from_factory = nullptr;
  if (compaction_filter == nullptr) {
    compaction_filter_from_factory =
        sub_compact->compaction->CreateCompactionFilter();
    compaction_filter = compaction_filter_from_factory.get();
  }
  if (compaction_filter != nullptr && !compaction_filter->IgnoreSnapshots()) {
    sub_compact->status = Status::NotSupported(
        "CompactionFilter::IgnoreSnapshots() = false is not supported "
        "anymore.");
    return;
  }

917 918
  CompactionRangeDelAggregator range_del_agg(&cfd->internal_comparator(),
                                             existing_snapshots_);
919 920 921 922 923 924 925 926
  ReadOptions read_options;
  read_options.verify_checksums = true;
  read_options.fill_cache = false;
  // Compaction iterators shouldn't be confined to a single prefix.
  // Compactions use Seek() for
  // (a) concurrent compactions,
  // (b) CompactionFilter::Decision::kRemoveAndSkipUntil.
  read_options.total_order_seek = true;
927 928 929

  // Although the v2 aggregator is what the level iterator(s) know about,
  // the AddTombstones calls will be propagated down to the v1 aggregator.
930 931 932
  std::unique_ptr<InternalIterator> input(
      versions_->MakeInputIterator(read_options, sub_compact->compaction,
                                   &range_del_agg, file_options_for_read_));
933

934 935
  AutoThreadOperationStageUpdater stage_updater(
      ThreadStatus::STAGE_COMPACTION_PROCESS_KV);
936 937 938

  // I/O measurement variables
  PerfLevel prev_perf_level = PerfLevel::kEnableTime;
939
  const uint64_t kRecordStatsEvery = 1000;
940 941 942 943
  uint64_t prev_write_nanos = 0;
  uint64_t prev_fsync_nanos = 0;
  uint64_t prev_range_sync_nanos = 0;
  uint64_t prev_prepare_write_nanos = 0;
944 945
  uint64_t prev_cpu_write_nanos = 0;
  uint64_t prev_cpu_read_nanos = 0;
946 947
  if (measure_io_stats_) {
    prev_perf_level = GetPerfLevel();
948
    SetPerfLevel(PerfLevel::kEnableTimeAndCPUTimeExceptForMutex);
I
Igor Canadi 已提交
949 950 951 952
    prev_write_nanos = IOSTATS(write_nanos);
    prev_fsync_nanos = IOSTATS(fsync_nanos);
    prev_range_sync_nanos = IOSTATS(range_sync_nanos);
    prev_prepare_write_nanos = IOSTATS(prepare_write_nanos);
953 954
    prev_cpu_write_nanos = IOSTATS(cpu_write_nanos);
    prev_cpu_read_nanos = IOSTATS(cpu_read_nanos);
955 956
  }

I
Igor Canadi 已提交
957 958 959 960 961
  MergeHelper merge(
      env_, cfd->user_comparator(), cfd->ioptions()->merge_operator,
      compaction_filter, db_options_.info_log.get(),
      false /* internal key corruption is expected */,
      existing_snapshots_.empty() ? 0 : existing_snapshots_.back(),
962
      snapshot_checker_, compact_->compaction->level(),
963
      db_options_.statistics.get());
I
Igor Canadi 已提交
964

965 966 967 968 969 970 971 972 973 974 975 976 977 978 979 980
  const MutableCFOptions* mutable_cf_options =
      sub_compact->compaction->mutable_cf_options();
  assert(mutable_cf_options);

  std::vector<std::string> blob_file_paths;

  std::unique_ptr<BlobFileBuilder> blob_file_builder(
      mutable_cf_options->enable_blob_files
          ? new BlobFileBuilder(
                versions_, env_, fs_.get(),
                sub_compact->compaction->immutable_cf_options(),
                mutable_cf_options, &file_options_, job_id_, cfd->GetID(),
                cfd->GetName(), Env::IOPriority::IO_LOW, write_hint_,
                &blob_file_paths, &sub_compact->blob_file_additions)
          : nullptr);

981
  TEST_SYNC_POINT("CompactionJob::Run():Inprogress");
982 983 984
  TEST_SYNC_POINT_CALLBACK(
      "CompactionJob::Run():PausingManualCompaction:1",
      reinterpret_cast<void*>(
985
          const_cast<std::atomic<int>*>(manual_compaction_paused_)));
986

987 988
  Slice* start = sub_compact->start;
  Slice* end = sub_compact->end;
989 990 991
  if (start != nullptr) {
    IterKey start_iter;
    start_iter.SetInternalKey(*start, kMaxSequenceNumber, kValueTypeForSeek);
992
    input->Seek(start_iter.GetInternalKey());
993 994 995 996
  } else {
    input->SeekToFirst();
  }

997 998 999
  Status status;
  sub_compact->c_iter.reset(new CompactionIterator(
      input.get(), cfd->user_comparator(), &merge, versions_->LastSequence(),
Y
Yi Wu 已提交
1000
      &existing_snapshots_, earliest_write_conflict_snapshot_,
1001 1002
      snapshot_checker_, env_, ShouldReportDetailedTime(env_, stats_),
      /*expect_valid_internal_key=*/true, &range_del_agg,
1003
      blob_file_builder.get(), db_options_.allow_data_in_errors,
1004 1005 1006
      sub_compact->compaction, compaction_filter, shutting_down_,
      preserve_deletes_seqnum_, manual_compaction_paused_,
      db_options_.info_log));
1007 1008
  auto c_iter = sub_compact->c_iter.get();
  c_iter->SeekToFirst();
1009
  if (c_iter->Valid() && sub_compact->compaction->output_level() != 0) {
1010 1011 1012
    // ShouldStopBefore() maintains state based on keys processed so far. The
    // compaction loop always calls it on the "next" key, thus won't tell it the
    // first key. So we do that here.
1013 1014
    sub_compact->ShouldStopBefore(c_iter->key(),
                                  sub_compact->current_output_file_size);
1015
  }
1016
  const auto& c_iter_stats = c_iter->iter_stats();
1017

1018 1019 1020 1021 1022 1023
  std::unique_ptr<SstPartitioner> partitioner =
      sub_compact->compaction->output_level() == 0
          ? nullptr
          : sub_compact->compaction->CreateSstPartitioner();
  std::string last_key_for_partitioner;

1024
  while (status.ok() && !cfd->IsDropped() && c_iter->Valid()) {
1025 1026 1027 1028
    // Invariant: c_iter.status() is guaranteed to be OK if c_iter->Valid()
    // returns true.
    const Slice& key = c_iter->key();
    const Slice& value = c_iter->value();
1029

1030 1031
    // If an end key (exclusive) is specified, check if the current key is
    // >= than it and exit if it is because the iterator is out of its range
1032
    if (end != nullptr &&
1033
        cfd->user_comparator()->Compare(c_iter->user_key(), *end) >= 0) {
1034
      break;
I
Igor Canadi 已提交
1035
    }
1036 1037 1038 1039 1040
    if (c_iter_stats.num_input_records % kRecordStatsEvery ==
        kRecordStatsEvery - 1) {
      RecordDroppedKeys(c_iter_stats, &sub_compact->compaction_job_stats);
      c_iter->ResetRecordCounts();
      RecordCompactionIOStats();
I
Igor Canadi 已提交
1041 1042
    }

1043 1044 1045 1046
    // Open output file if necessary
    if (sub_compact->builder == nullptr) {
      status = OpenCompactionOutputFile(sub_compact);
      if (!status.ok()) {
1047 1048
        break;
      }
1049
    }
1050 1051 1052 1053
    status = sub_compact->AddToBuilder(key, value);
    if (!status.ok()) {
      break;
    }
1054

1055 1056
    sub_compact->current_output_file_size =
        sub_compact->builder->EstimatedFileSize();
1057
    const ParsedInternalKey& ikey = c_iter->ikey();
1058
    sub_compact->current_output()->meta.UpdateBoundaries(
1059
        key, value, ikey.sequence, ikey.type);
1060 1061
    sub_compact->num_output_records++;

1062 1063 1064 1065
    // Close output file if it is big enough. Two possibilities determine it's
    // time to close it: (1) the current key should be this file's last key, (2)
    // the next key should not be in this file.
    //
1066 1067 1068 1069
    // TODO(aekmekji): determine if file should be closed earlier than this
    // during subcompactions (i.e. if output size, estimated by input size, is
    // going to be 1.2MB and max_output_file_size = 1MB, prefer to have 0.6MB
    // and 0.6MB instead of 1MB and 0.2MB)
1070
    bool output_file_ended = false;
1071 1072 1073
    if (sub_compact->compaction->output_level() != 0 &&
        sub_compact->current_output_file_size >=
            sub_compact->compaction->max_output_file_size()) {
1074 1075 1076 1077
      // (1) this key terminates the file. For historical reasons, the iterator
      // status before advancing will be given to FinishCompactionOutputFile().
      output_file_ended = true;
    }
1078 1079 1080
    TEST_SYNC_POINT_CALLBACK(
        "CompactionJob::Run():PausingManualCompaction:2",
        reinterpret_cast<void*>(
1081
            const_cast<std::atomic<int>*>(manual_compaction_paused_)));
1082 1083 1084 1085
    if (partitioner.get()) {
      last_key_for_partitioner.assign(c_iter->user_key().data_,
                                      c_iter->user_key().size_);
    }
1086
    c_iter->Next();
1087 1088 1089
    if (c_iter->status().IsManualCompactionPaused()) {
      break;
    }
1090 1091 1092 1093 1094 1095 1096 1097 1098 1099 1100 1101 1102 1103
    if (!output_file_ended && c_iter->Valid()) {
      if (((partitioner.get() &&
            partitioner->ShouldPartition(PartitionerRequest(
                last_key_for_partitioner, c_iter->user_key(),
                sub_compact->current_output_file_size)) == kRequired) ||
           (sub_compact->compaction->output_level() != 0 &&
            sub_compact->ShouldStopBefore(
                c_iter->key(), sub_compact->current_output_file_size))) &&
          sub_compact->builder != nullptr) {
        // (2) this key belongs to the next file. For historical reasons, the
        // iterator status after advancing will be given to
        // FinishCompactionOutputFile().
        output_file_ended = true;
      }
1104 1105
    }
    if (output_file_ended) {
1106 1107 1108 1109
      const Slice* next_key = nullptr;
      if (c_iter->Valid()) {
        next_key = &c_iter->key();
      }
A
Andrew Kryczka 已提交
1110
      CompactionIterationStats range_del_out_stats;
1111 1112 1113
      status = FinishCompactionOutputFile(input->status(), sub_compact,
                                          &range_del_agg, &range_del_out_stats,
                                          next_key);
A
Andrew Kryczka 已提交
1114 1115
      RecordDroppedKeys(range_del_out_stats,
                        &sub_compact->compaction_job_stats);
I
Igor Canadi 已提交
1116 1117
    }
  }
1118

1119 1120 1121 1122
  sub_compact->compaction_job_stats.num_input_deletion_records =
      c_iter_stats.num_input_deletion_records;
  sub_compact->compaction_job_stats.num_corrupt_keys =
      c_iter_stats.num_input_corrupt_records;
1123 1124 1125 1126
  sub_compact->compaction_job_stats.num_single_del_fallthru =
      c_iter_stats.num_single_del_fallthru;
  sub_compact->compaction_job_stats.num_single_del_mismatch =
      c_iter_stats.num_single_del_mismatch;
1127 1128 1129 1130 1131 1132 1133 1134
  sub_compact->compaction_job_stats.total_input_raw_key_bytes +=
      c_iter_stats.total_input_raw_key_bytes;
  sub_compact->compaction_job_stats.total_input_raw_value_bytes +=
      c_iter_stats.total_input_raw_value_bytes;

  RecordTick(stats_, FILTER_OPERATION_TOTAL_TIME,
             c_iter_stats.total_filter_time);
  RecordDroppedKeys(c_iter_stats, &sub_compact->compaction_job_stats);
I
Igor Canadi 已提交
1135 1136
  RecordCompactionIOStats();

1137 1138 1139 1140 1141 1142 1143
  if (status.ok() && cfd->IsDropped()) {
    status =
        Status::ColumnFamilyDropped("Column family dropped during compaction");
  }
  if ((status.ok() || status.IsColumnFamilyDropped()) &&
      shutting_down_->load(std::memory_order_relaxed)) {
    status = Status::ShutdownInProgress("Database shutdown");
1144
  }
1145 1146
  if ((status.ok() || status.IsColumnFamilyDropped()) &&
      (manual_compaction_paused_ &&
1147
       manual_compaction_paused_->load(std::memory_order_relaxed) > 0)) {
1148 1149
    status = Status::Incomplete(Status::SubCode::kManualCompactionPaused);
  }
1150 1151 1152 1153 1154 1155 1156
  if (status.ok()) {
    status = input->status();
  }
  if (status.ok()) {
    status = c_iter->status();
  }

1157
  if (status.ok() && sub_compact->builder == nullptr &&
1158
      sub_compact->outputs.size() == 0 && !range_del_agg.IsEmpty()) {
1159
    // handle subcompaction containing only range deletions
1160 1161
    status = OpenCompactionOutputFile(sub_compact);
  }
1162 1163 1164 1165

  // Call FinishCompactionOutputFile() even if status is not ok: it needs to
  // close the output file.
  if (sub_compact->builder != nullptr) {
A
Andrew Kryczka 已提交
1166
    CompactionIterationStats range_del_out_stats;
1167
    Status s = FinishCompactionOutputFile(status, sub_compact, &range_del_agg,
1168
                                          &range_del_out_stats);
1169
    if (!s.ok() && status.ok()) {
1170 1171
      status = s;
    }
A
Andrew Kryczka 已提交
1172
    RecordDroppedKeys(range_del_out_stats, &sub_compact->compaction_job_stats);
1173 1174
  }

1175 1176 1177 1178 1179 1180 1181 1182
  if (blob_file_builder) {
    if (status.ok()) {
      status = blob_file_builder->Finish();
    }

    blob_file_builder.reset();
  }

1183 1184 1185
  sub_compact->compaction_job_stats.cpu_micros =
      env_->NowCPUNanos() / 1000 - prev_cpu_micros;

1186 1187
  if (measure_io_stats_) {
    sub_compact->compaction_job_stats.file_write_nanos +=
I
Igor Canadi 已提交
1188
        IOSTATS(write_nanos) - prev_write_nanos;
1189
    sub_compact->compaction_job_stats.file_fsync_nanos +=
I
Igor Canadi 已提交
1190
        IOSTATS(fsync_nanos) - prev_fsync_nanos;
1191
    sub_compact->compaction_job_stats.file_range_sync_nanos +=
I
Igor Canadi 已提交
1192
        IOSTATS(range_sync_nanos) - prev_range_sync_nanos;
1193
    sub_compact->compaction_job_stats.file_prepare_write_nanos +=
I
Igor Canadi 已提交
1194
        IOSTATS(prepare_write_nanos) - prev_prepare_write_nanos;
1195
    sub_compact->compaction_job_stats.cpu_micros -=
1196 1197 1198
        (IOSTATS(cpu_write_nanos) - prev_cpu_write_nanos +
         IOSTATS(cpu_read_nanos) - prev_cpu_read_nanos) /
        1000;
1199
    if (prev_perf_level != PerfLevel::kEnableTimeAndCPUTimeExceptForMutex) {
1200 1201 1202
      SetPerfLevel(prev_perf_level);
    }
  }
1203 1204 1205 1206 1207 1208 1209 1210 1211 1212
#ifdef ROCKSDB_ASSERT_STATUS_CHECKED
  if (!status.ok()) {
    if (sub_compact->c_iter) {
      sub_compact->c_iter->status().PermitUncheckedError();
    }
    if (input) {
      input->status().PermitUncheckedError();
    }
  }
#endif  // ROCKSDB_ASSERT_STATUS_CHECKED
1213

1214 1215
  sub_compact->c_iter.reset();
  input.reset();
1216
  sub_compact->status = status;
I
Igor Canadi 已提交
1217 1218
}

1219
void CompactionJob::RecordDroppedKeys(
A
Andrew Kryczka 已提交
1220
    const CompactionIterationStats& c_iter_stats,
1221
    CompactionJobStats* compaction_job_stats) {
1222 1223 1224
  if (c_iter_stats.num_record_drop_user > 0) {
    RecordTick(stats_, COMPACTION_KEY_DROP_USER,
               c_iter_stats.num_record_drop_user);
1225
  }
1226 1227 1228
  if (c_iter_stats.num_record_drop_hidden > 0) {
    RecordTick(stats_, COMPACTION_KEY_DROP_NEWER_ENTRY,
               c_iter_stats.num_record_drop_hidden);
1229
    if (compaction_job_stats) {
1230 1231
      compaction_job_stats->num_records_replaced +=
          c_iter_stats.num_record_drop_hidden;
1232 1233
    }
  }
1234 1235 1236
  if (c_iter_stats.num_record_drop_obsolete > 0) {
    RecordTick(stats_, COMPACTION_KEY_DROP_OBSOLETE,
               c_iter_stats.num_record_drop_obsolete);
1237
    if (compaction_job_stats) {
1238 1239
      compaction_job_stats->num_expired_deletion_records +=
          c_iter_stats.num_record_drop_obsolete;
1240
    }
1241
  }
A
Andrew Kryczka 已提交
1242 1243 1244 1245 1246 1247 1248 1249
  if (c_iter_stats.num_record_drop_range_del > 0) {
    RecordTick(stats_, COMPACTION_KEY_DROP_RANGE_DEL,
               c_iter_stats.num_record_drop_range_del);
  }
  if (c_iter_stats.num_range_del_drop_obsolete > 0) {
    RecordTick(stats_, COMPACTION_RANGE_DEL_DROP_OBSOLETE,
               c_iter_stats.num_range_del_drop_obsolete);
  }
1250 1251 1252 1253
  if (c_iter_stats.num_optimized_del_drop_obsolete > 0) {
    RecordTick(stats_, COMPACTION_OPTIMIZED_DEL_DROP_OBSOLETE,
               c_iter_stats.num_optimized_del_drop_obsolete);
  }
1254 1255
}

1256
Status CompactionJob::FinishCompactionOutputFile(
1257
    const Status& input_status, SubcompactionState* sub_compact,
1258
    CompactionRangeDelAggregator* range_del_agg,
A
Andrew Kryczka 已提交
1259
    CompactionIterationStats* range_del_out_stats,
1260
    const Slice* next_table_min_key /* = nullptr */) {
1261 1262
  AutoThreadOperationStageUpdater stage_updater(
      ThreadStatus::STAGE_COMPACTION_SYNC_FILE);
1263 1264 1265
  assert(sub_compact != nullptr);
  assert(sub_compact->outfile);
  assert(sub_compact->builder != nullptr);
1266
  assert(sub_compact->current_output() != nullptr);
I
Igor Canadi 已提交
1267

1268
  uint64_t output_number = sub_compact->current_output()->meta.fd.GetNumber();
I
Igor Canadi 已提交
1269 1270
  assert(output_number != 0);

1271 1272
  ColumnFamilyData* cfd = sub_compact->compaction->column_family_data();
  const Comparator* ucmp = cfd->user_comparator();
1273 1274
  std::string file_checksum = kUnknownFileChecksum;
  std::string file_checksum_func_name = kUnknownFileChecksumFuncName;
1275

I
Igor Canadi 已提交
1276
  // Check for iterator errors
1277
  Status s = input_status;
1278
  auto meta = &sub_compact->current_output()->meta;
1279
  assert(meta != nullptr);
1280
  if (s.ok()) {
1281
    Slice lower_bound_guard, upper_bound_guard;
Z
zhangjinpeng1987 已提交
1282
    std::string smallest_user_key;
1283
    const Slice *lower_bound, *upper_bound;
1284
    bool lower_bound_from_sub_compact = false;
1285 1286 1287 1288
    if (sub_compact->outputs.size() == 1) {
      // For the first output table, include range tombstones before the min key
      // but after the subcompaction boundary.
      lower_bound = sub_compact->start;
1289
      lower_bound_from_sub_compact = true;
1290 1291 1292 1293
    } else if (meta->smallest.size() > 0) {
      // For subsequent output tables, only include range tombstones from min
      // key onwards since the previous file was extended to contain range
      // tombstones falling before min key.
Z
zhangjinpeng1987 已提交
1294 1295
      smallest_user_key = meta->smallest.user_key().ToString(false /*hex*/);
      lower_bound_guard = Slice(smallest_user_key);
1296 1297 1298 1299 1300
      lower_bound = &lower_bound_guard;
    } else {
      lower_bound = nullptr;
    }
    if (next_table_min_key != nullptr) {
1301 1302 1303 1304 1305 1306
      // This may be the last file in the subcompaction in some cases, so we
      // need to compare the end key of subcompaction with the next file start
      // key. When the end key is chosen by the subcompaction, we know that
      // it must be the biggest key in output file. Therefore, it is safe to
      // use the smaller key as the upper bound of the output file, to ensure
      // that there is no overlapping between different output files.
1307
      upper_bound_guard = ExtractUserKey(*next_table_min_key);
1308 1309 1310 1311 1312 1313
      if (sub_compact->end != nullptr &&
          ucmp->Compare(upper_bound_guard, *sub_compact->end) >= 0) {
        upper_bound = sub_compact->end;
      } else {
        upper_bound = &upper_bound_guard;
      }
1314 1315 1316 1317 1318
    } else {
      // This is the last file in the subcompaction, so extend until the
      // subcompaction ends.
      upper_bound = sub_compact->end;
    }
1319 1320 1321 1322
    auto earliest_snapshot = kMaxSequenceNumber;
    if (existing_snapshots_.size() > 0) {
      earliest_snapshot = existing_snapshots_[0];
    }
1323 1324 1325 1326 1327 1328 1329
    bool has_overlapping_endpoints;
    if (upper_bound != nullptr && meta->largest.size() > 0) {
      has_overlapping_endpoints =
          ucmp->Compare(meta->largest.user_key(), *upper_bound) == 0;
    } else {
      has_overlapping_endpoints = false;
    }
1330

1331 1332 1333 1334 1335 1336 1337
    // The end key of the subcompaction must be bigger or equal to the upper
    // bound. If the end of subcompaction is null or the upper bound is null,
    // it means that this file is the last file in the compaction. So there
    // will be no overlapping between this file and others.
    assert(sub_compact->end == nullptr ||
           upper_bound == nullptr ||
           ucmp->Compare(*upper_bound , *sub_compact->end) <= 0);
1338 1339 1340 1341 1342 1343 1344 1345 1346 1347
    auto it = range_del_agg->NewIterator(lower_bound, upper_bound,
                                         has_overlapping_endpoints);
    // Position the range tombstone output iterator. There may be tombstone
    // fragments that are entirely out of range, so make sure that we do not
    // include those.
    if (lower_bound != nullptr) {
      it->Seek(*lower_bound);
    } else {
      it->SeekToFirst();
    }
1348
    TEST_SYNC_POINT("CompactionJob::FinishCompactionOutputFile1");
1349 1350
    for (; it->Valid(); it->Next()) {
      auto tombstone = it->Tombstone();
1351 1352 1353 1354 1355 1356 1357 1358 1359 1360 1361 1362
      if (upper_bound != nullptr) {
        int cmp = ucmp->Compare(*upper_bound, tombstone.start_key_);
        if ((has_overlapping_endpoints && cmp < 0) ||
            (!has_overlapping_endpoints && cmp <= 0)) {
          // Tombstones starting after upper_bound only need to be included in
          // the next table. If the current SST ends before upper_bound, i.e.,
          // `has_overlapping_endpoints == false`, we can also skip over range
          // tombstones that start exactly at upper_bound. Such range tombstones
          // will be included in the next file and are not relevant to the point
          // keys or endpoints of the current file.
          break;
        }
1363 1364 1365 1366 1367 1368 1369 1370 1371 1372 1373
      }

      if (bottommost_level_ && tombstone.seq_ <= earliest_snapshot) {
        // TODO(andrewkr): tombstones that span multiple output files are
        // counted for each compaction output file, so lots of double counting.
        range_del_out_stats->num_range_del_drop_obsolete++;
        range_del_out_stats->num_record_drop_obsolete++;
        continue;
      }

      auto kv = tombstone.Serialize();
1374 1375
      assert(lower_bound == nullptr ||
             ucmp->Compare(*lower_bound, kv.second) < 0);
1376 1377
      // Range tombstone is not supported by output validator yet.
      sub_compact->builder->Add(kv.first.Encode(), kv.second);
1378 1379 1380 1381 1382 1383 1384
      InternalKey smallest_candidate = std::move(kv.first);
      if (lower_bound != nullptr &&
          ucmp->Compare(smallest_candidate.user_key(), *lower_bound) <= 0) {
        // Pretend the smallest key has the same user key as lower_bound
        // (the max key in the previous table or subcompaction) in order for
        // files to appear key-space partitioned.
        //
1385 1386 1387 1388 1389 1390 1391 1392 1393 1394 1395 1396 1397 1398 1399 1400 1401 1402
        // When lower_bound is chosen by a subcompaction, we know that
        // subcompactions over smaller keys cannot contain any keys at
        // lower_bound. We also know that smaller subcompactions exist, because
        // otherwise the subcompaction woud be unbounded on the left. As a
        // result, we know that no other files on the output level will contain
        // actual keys at lower_bound (an output file may have a largest key of
        // lower_bound@kMaxSequenceNumber, but this only indicates a large range
        // tombstone was truncated). Therefore, it is safe to use the
        // tombstone's sequence number, to ensure that keys at lower_bound at
        // lower levels are covered by truncated tombstones.
        //
        // If lower_bound was chosen by the smallest data key in the file,
        // choose lowest seqnum so this file's smallest internal key comes after
        // the previous file's largest. The fake seqnum is OK because the read
        // path's file-picking code only considers user key.
        smallest_candidate = InternalKey(
            *lower_bound, lower_bound_from_sub_compact ? tombstone.seq_ : 0,
            kTypeRangeDeletion);
1403 1404 1405 1406 1407 1408 1409 1410 1411 1412 1413 1414 1415 1416 1417 1418 1419 1420 1421 1422 1423
      }
      InternalKey largest_candidate = tombstone.SerializeEndKey();
      if (upper_bound != nullptr &&
          ucmp->Compare(*upper_bound, largest_candidate.user_key()) <= 0) {
        // Pretend the largest key has the same user key as upper_bound (the
        // min key in the following table or subcompaction) in order for files
        // to appear key-space partitioned.
        //
        // Choose highest seqnum so this file's largest internal key comes
        // before the next file's/subcompaction's smallest. The fake seqnum is
        // OK because the read path's file-picking code only considers the user
        // key portion.
        //
        // Note Seek() also creates InternalKey with (user_key,
        // kMaxSequenceNumber), but with kTypeDeletion (0x7) instead of
        // kTypeRangeDeletion (0xF), so the range tombstone comes before the
        // Seek() key in InternalKey's ordering. So Seek() will look in the
        // next file for the user key.
        largest_candidate =
            InternalKey(*upper_bound, kMaxSequenceNumber, kTypeRangeDeletion);
      }
1424 1425 1426 1427 1428 1429
#ifndef NDEBUG
      SequenceNumber smallest_ikey_seqnum = kMaxSequenceNumber;
      if (meta->smallest.size() > 0) {
        smallest_ikey_seqnum = GetInternalKeySeqno(meta->smallest.Encode());
      }
#endif
1430 1431 1432
      meta->UpdateBoundariesForRange(smallest_candidate, largest_candidate,
                                     tombstone.seq_,
                                     cfd->internal_comparator());
1433 1434 1435 1436 1437 1438 1439 1440

      // The smallest key in a file is used for range tombstone truncation, so
      // it cannot have a seqnum of 0 (unless the smallest data key in a file
      // has a seqnum of 0). Otherwise, the truncated tombstone may expose
      // deleted keys at lower levels.
      assert(smallest_ikey_seqnum == 0 ||
             ExtractInternalKeyFooter(meta->smallest.Encode()) !=
                 PackSequenceAndType(0, kTypeRangeDeletion));
1441
    }
S
Siying Dong 已提交
1442
    meta->marked_for_compaction = sub_compact->builder->NeedCompact();
1443
  }
1444
  const uint64_t current_entries = sub_compact->builder->NumEntries();
I
Igor Canadi 已提交
1445
  if (s.ok()) {
1446
    s = sub_compact->builder->Finish();
I
Igor Canadi 已提交
1447
  } else {
1448
    sub_compact->builder->Abandon();
I
Igor Canadi 已提交
1449
  }
1450 1451 1452
  IOStatus io_s = sub_compact->builder->io_status();
  if (s.ok()) {
    s = io_s;
1453
  }
1454
  const uint64_t current_bytes = sub_compact->builder->FileSize();
S
Siying Dong 已提交
1455 1456 1457
  if (s.ok()) {
    meta->fd.file_size = current_bytes;
  }
1458
  sub_compact->current_output()->finished = true;
1459
  sub_compact->total_bytes += current_bytes;
I
Igor Canadi 已提交
1460 1461

  // Finish and check for file errors
S
Sagar Vemuri 已提交
1462
  if (s.ok()) {
1463
    StopWatch sw(env_, stats_, COMPACTION_OUTFILE_SYNC_MICROS);
1464
    io_s = sub_compact->outfile->Sync(db_options_.use_fsync);
I
Igor Canadi 已提交
1465
  }
1466
  if (s.ok() && io_s.ok()) {
1467 1468
    io_s = sub_compact->outfile->Close();
  }
1469
  if (s.ok() && io_s.ok()) {
1470 1471 1472 1473
    // Add the checksum information to file metadata.
    meta->file_checksum = sub_compact->outfile->GetFileChecksum();
    meta->file_checksum_func_name =
        sub_compact->outfile->GetFileChecksumFuncName();
1474 1475
    file_checksum = meta->file_checksum;
    file_checksum_func_name = meta->file_checksum_func_name;
1476
  }
1477
  if (s.ok()) {
1478
    s = io_s;
I
Igor Canadi 已提交
1479
  }
1480 1481
  if (sub_compact->io_status.ok()) {
    sub_compact->io_status = io_s;
1482 1483 1484
    // Since this error is really a copy of the
    // "normal" status, it does not also need to be checked
    sub_compact->io_status.PermitUncheckedError();
1485
  }
1486
  sub_compact->outfile.reset();
I
Igor Canadi 已提交
1487

1488 1489 1490 1491 1492 1493
  TableProperties tp;
  if (s.ok()) {
    tp = sub_compact->builder->GetTableProperties();
  }

  if (s.ok() && current_entries == 0 && tp.num_range_deletions == 0) {
Z
zhangjinpeng1987 已提交
1494 1495 1496
    // If there is nothing to output, no necessary to generate a sst file.
    // This happens when the output level is bottom level, at the same time
    // the sub_compact output nothing.
1497 1498 1499
    std::string fname =
        TableFileName(sub_compact->compaction->immutable_cf_options()->cf_paths,
                      meta->fd.GetNumber(), meta->fd.GetPathId());
Z
zhangjinpeng1987 已提交
1500 1501 1502 1503 1504 1505
    env_->DeleteFile(fname);

    // Also need to remove the file from outputs, or it will be added to the
    // VersionEdit.
    assert(!sub_compact->outputs.empty());
    sub_compact->outputs.pop_back();
1506
    meta = nullptr;
Z
zhangjinpeng1987 已提交
1507 1508
  }

1509
  if (s.ok() && (current_entries > 0 || tp.num_range_deletions > 0)) {
1510
    // Output to event logger and fire events.
1511 1512 1513 1514 1515 1516 1517 1518
    sub_compact->current_output()->table_properties =
        std::make_shared<TableProperties>(tp);
    ROCKS_LOG_INFO(db_options_.info_log,
                   "[%s] [JOB %d] Generated table #%" PRIu64 ": %" PRIu64
                   " keys, %" PRIu64 " bytes%s",
                   cfd->GetName().c_str(), job_id_, output_number,
                   current_entries, current_bytes,
                   meta->marked_for_compaction ? " (need compaction)" : "");
I
Igor Canadi 已提交
1519
  }
S
Siying Dong 已提交
1520 1521
  std::string fname;
  FileDescriptor output_fd;
1522
  uint64_t oldest_blob_file_number = kInvalidBlobFileNumber;
S
Siying Dong 已提交
1523
  if (meta != nullptr) {
1524 1525 1526
    fname =
        TableFileName(sub_compact->compaction->immutable_cf_options()->cf_paths,
                      meta->fd.GetNumber(), meta->fd.GetPathId());
S
Siying Dong 已提交
1527
    output_fd = meta->fd;
1528
    oldest_blob_file_number = meta->oldest_blob_file_number;
S
Siying Dong 已提交
1529 1530 1531
  } else {
    fname = "(nil)";
  }
1532 1533
  EventHelpers::LogAndNotifyTableFileCreationFinished(
      event_logger_, cfd->ioptions()->listeners, dbname_, cfd->GetName(), fname,
1534
      job_id_, output_fd, oldest_blob_file_number, tp,
1535 1536
      TableFileCreationReason::kCompaction, s, file_checksum,
      file_checksum_func_name);
1537

1538
#ifndef ROCKSDB_LITE
1539 1540 1541
  // Report new file to SstFileManagerImpl
  auto sfm =
      static_cast<SstFileManagerImpl*>(db_options_.sst_file_manager.get());
S
Siying Dong 已提交
1542
  if (sfm && meta != nullptr && meta->fd.GetPathId() == 0) {
1543 1544 1545 1546
    Status add_s = sfm->OnAddFile(fname);
    if (!add_s.ok() && s.ok()) {
      s = add_s;
    }
1547
    if (sfm->IsMaxAllowedSpaceReached()) {
1548 1549
      // TODO(ajkr): should we return OK() if max space was reached by the final
      // compaction output file (similarly to how flush works when full)?
1550
      s = Status::SpaceLimit("Max allowed space was reached");
1551 1552 1553
      TEST_SYNC_POINT(
          "CompactionJob::FinishCompactionOutputFile:"
          "MaxAllowedSpaceReached");
1554
      InstrumentedMutexLock l(db_mutex_);
1555 1556 1557
      // Should handle return error?
      db_error_handler_->SetBGError(s, BackgroundErrorReason::kCompaction)
          .PermitUncheckedError();
1558 1559
    }
  }
1560
#endif
1561

1562
  sub_compact->builder.reset();
1563
  sub_compact->current_output_file_size = 0;
I
Igor Canadi 已提交
1564 1565 1566
  return s;
}

I
Igor Canadi 已提交
1567
Status CompactionJob::InstallCompactionResults(
1568
    const MutableCFOptions& mutable_cf_options) {
1569 1570
  assert(compact_);

1571
  db_mutex_->AssertHeld();
I
Igor Canadi 已提交
1572

1573
  auto* compaction = compact_->compaction;
1574 1575
  assert(compaction);

I
Igor Canadi 已提交
1576 1577 1578 1579
  // paranoia: verify that the files that we started with
  // still exist in the current version and in the same original level.
  // This ensures that a concurrent compaction did not erroneously
  // pick the same files to compact_.
1580 1581 1582
  if (!versions_->VerifyCompactionFileConsistency(compaction)) {
    Compaction::InputLevelSummaryBuffer inputs_summary;

1583 1584 1585
    ROCKS_LOG_ERROR(db_options_.info_log, "[%s] [JOB %d] Compaction %s aborted",
                    compaction->column_family_data()->GetName().c_str(),
                    job_id_, compaction->InputLevelSummary(&inputs_summary));
I
Igor Canadi 已提交
1586 1587 1588
    return Status::Corruption("Compaction input files inconsistent");
  }

1589 1590
  {
    Compaction::InputLevelSummaryBuffer inputs_summary;
1591 1592 1593 1594 1595
    ROCKS_LOG_INFO(db_options_.info_log,
                   "[%s] [JOB %d] Compacted %s => %" PRIu64 " bytes",
                   compaction->column_family_data()->GetName().c_str(), job_id_,
                   compaction->InputLevelSummary(&inputs_summary),
                   compact_->total_bytes + compact_->total_blob_bytes);
1596
  }
I
Igor Canadi 已提交
1597

1598 1599 1600
  VersionEdit* const edit = compaction->edit();
  assert(edit);

G
Gihwan Oh 已提交
1601
  // Add compaction inputs
1602
  compaction->AddInputDeletions(edit);
1603

1604 1605
  for (const auto& sub_compact : compact_->sub_compact_states) {
    for (const auto& out : sub_compact.outputs) {
1606 1607 1608 1609 1610
      edit->AddFile(compaction->output_level(), out.meta);
    }

    for (const auto& blob : sub_compact.blob_file_additions) {
      edit->AddBlobFile(blob);
1611
    }
1612
  }
1613

1614
  return versions_->LogAndApply(compaction->column_family_data(),
1615 1616
                                mutable_cf_options, edit, db_mutex_,
                                db_directory_);
I
Igor Canadi 已提交
1617 1618 1619 1620
}

void CompactionJob::RecordCompactionIOStats() {
  RecordTick(stats_, COMPACT_READ_BYTES, IOSTATS(bytes_read));
1621 1622 1623 1624 1625 1626 1627 1628 1629 1630 1631 1632 1633
  RecordTick(stats_, COMPACT_WRITE_BYTES, IOSTATS(bytes_written));
  CompactionReason compaction_reason =
      compact_->compaction->compaction_reason();
  if (compaction_reason == CompactionReason::kFilesMarkedForCompaction) {
    RecordTick(stats_, COMPACT_READ_BYTES_MARKED, IOSTATS(bytes_read));
    RecordTick(stats_, COMPACT_WRITE_BYTES_MARKED, IOSTATS(bytes_written));
  } else if (compaction_reason == CompactionReason::kPeriodicCompaction) {
    RecordTick(stats_, COMPACT_READ_BYTES_PERIODIC, IOSTATS(bytes_read));
    RecordTick(stats_, COMPACT_WRITE_BYTES_PERIODIC, IOSTATS(bytes_written));
  } else if (compaction_reason == CompactionReason::kTtl) {
    RecordTick(stats_, COMPACT_READ_BYTES_TTL, IOSTATS(bytes_read));
    RecordTick(stats_, COMPACT_WRITE_BYTES_TTL, IOSTATS(bytes_written));
  }
1634 1635
  ThreadStatusUtil::IncreaseThreadOperationProperty(
      ThreadStatus::COMPACTION_BYTES_READ, IOSTATS(bytes_read));
I
Igor Canadi 已提交
1636
  IOSTATS_RESET(bytes_read);
1637 1638
  ThreadStatusUtil::IncreaseThreadOperationProperty(
      ThreadStatus::COMPACTION_BYTES_WRITTEN, IOSTATS(bytes_written));
I
Igor Canadi 已提交
1639 1640 1641
  IOSTATS_RESET(bytes_written);
}

1642 1643
Status CompactionJob::OpenCompactionOutputFile(
    SubcompactionState* sub_compact) {
1644 1645
  assert(sub_compact != nullptr);
  assert(sub_compact->builder == nullptr);
1646 1647
  // no need to lock because VersionSet::next_file_number_ is atomic
  uint64_t file_number = versions_->NewFileNumber();
1648 1649 1650
  std::string fname =
      TableFileName(sub_compact->compaction->immutable_cf_options()->cf_paths,
                    file_number, sub_compact->compaction->output_path_id());
1651 1652 1653 1654 1655 1656 1657 1658
  // Fire events.
  ColumnFamilyData* cfd = sub_compact->compaction->column_family_data();
#ifndef ROCKSDB_LITE
  EventHelpers::NotifyTableFileCreationStarted(
      cfd->ioptions()->listeners, dbname_, cfd->GetName(), fname, job_id_,
      TableFileCreationReason::kCompaction);
#endif  // !ROCKSDB_LITE
  // Make the output file
1659
  std::unique_ptr<FSWritableFile> writable_file;
A
Aliaksei Sandryhaila 已提交
1660
#ifndef NDEBUG
1661
  bool syncpoint_arg = file_options_.use_direct_writes;
1662
  TEST_SYNC_POINT_CALLBACK("CompactionJob::OpenCompactionOutputFile",
1663
                           &syncpoint_arg);
A
Aliaksei Sandryhaila 已提交
1664
#endif
1665
  Status s;
1666 1667
  IOStatus io_s =
      NewWritableFile(fs_.get(), fname, &writable_file, file_options_);
1668 1669 1670
  s = io_s;
  if (sub_compact->io_status.ok()) {
    sub_compact->io_status = io_s;
1671 1672 1673
    // Since this error is really a copy of the io_s that is checked below as s,
    // it does not also need to be checked.
    sub_compact->io_status.PermitUncheckedError();
1674
  }
I
Igor Canadi 已提交
1675
  if (!s.ok()) {
1676 1677
    ROCKS_LOG_ERROR(
        db_options_.info_log,
1678
        "[%s] [JOB %d] OpenCompactionOutputFiles for table #%" PRIu64
1679
        " fails at NewWritableFile with status %s",
1680 1681
        sub_compact->compaction->column_family_data()->GetName().c_str(),
        job_id_, file_number, s.ToString().c_str());
I
Igor Canadi 已提交
1682
    LogFlush(db_options_.info_log);
1683 1684
    EventHelpers::LogAndNotifyTableFileCreationFinished(
        event_logger_, cfd->ioptions()->listeners, dbname_, cfd->GetName(),
1685
        fname, job_id_, FileDescriptor(), kInvalidBlobFileNumber,
1686 1687
        TableProperties(), TableFileCreationReason::kCompaction, s,
        kUnknownFileChecksum, kUnknownFileChecksumFuncName);
I
Igor Canadi 已提交
1688 1689
    return s;
  }
1690

1691 1692 1693 1694 1695 1696 1697 1698 1699 1700 1701 1702 1703 1704 1705 1706 1707 1708
  // Try to figure out the output file's oldest ancester time.
  int64_t temp_current_time = 0;
  auto get_time_status = env_->GetCurrentTime(&temp_current_time);
  // Safe to proceed even if GetCurrentTime fails. So, log and proceed.
  if (!get_time_status.ok()) {
    ROCKS_LOG_WARN(db_options_.info_log,
                   "Failed to get current time. Status: %s",
                   get_time_status.ToString().c_str());
  }
  uint64_t current_time = static_cast<uint64_t>(temp_current_time);
  uint64_t oldest_ancester_time =
      sub_compact->compaction->MinInputFileOldestAncesterTime();
  if (oldest_ancester_time == port::kMaxUint64) {
    oldest_ancester_time = current_time;
  }

  // Initialize a SubcompactionState::Output and add it to sub_compact->outputs
  {
1709 1710 1711 1712 1713 1714 1715 1716 1717 1718 1719
    FileMetaData meta;
    meta.fd = FileDescriptor(file_number,
                             sub_compact->compaction->output_path_id(), 0);
    meta.oldest_ancester_time = oldest_ancester_time;
    meta.file_creation_time = current_time;
    sub_compact->outputs.emplace_back(
        std::move(meta), cfd->internal_comparator(),
        /*enable_order_check=*/
        sub_compact->compaction->mutable_cf_options()
            ->check_flush_compaction_key_order,
        /*enable_hash=*/paranoid_file_checks_);
1720
  }
I
Igor Canadi 已提交
1721

1722
  writable_file->SetIOPriority(Env::IOPriority::IO_LOW);
S
Stream  
Shaohua Li 已提交
1723
  writable_file->SetWriteLifeTimeHint(write_hint_);
1724 1725
  writable_file->SetPreallocationBlockSize(static_cast<size_t>(
      sub_compact->compaction->OutputFilePreallocationSize()));
1726 1727
  const auto& listeners =
      sub_compact->compaction->immutable_cf_options()->listeners;
1728 1729 1730 1731
  sub_compact->outfile.reset(new WritableFileWriter(
      std::move(writable_file), fname, file_options_, env_, io_tracer_,
      db_options_.statistics.get(), listeners,
      db_options_.file_checksum_gen_factory.get()));
I
Igor Canadi 已提交
1732

1733 1734 1735
  // If the Column family flag is to only optimize filters for hits,
  // we can skip creating filters if this is the bottommost_level where
  // data is going to be found
1736 1737
  bool skip_filters =
      cfd->ioptions()->optimize_filters_for_hits && bottommost_level_;
S
Sagar Vemuri 已提交
1738

1739
  sub_compact->builder.reset(NewTableBuilder(
1740 1741 1742 1743
      *cfd->ioptions(), *(sub_compact->compaction->mutable_cf_options()),
      cfd->internal_comparator(), cfd->int_tbl_prop_collector_factories(),
      cfd->GetID(), cfd->GetName(), sub_compact->outfile.get(),
      sub_compact->compaction->output_compression(),
1744
      0 /*sample_for_compression */,
1745
      sub_compact->compaction->output_compression_opts(),
1746 1747
      sub_compact->compaction->output_level(), skip_filters,
      oldest_ancester_time, 0 /* oldest_key_time */,
1748 1749
      sub_compact->compaction->max_output_file_size(), current_time, db_id_,
      db_session_id_));
I
Igor Canadi 已提交
1750 1751 1752 1753
  LogFlush(db_options_.info_log);
  return s;
}

1754
void CompactionJob::CleanupCompaction() {
1755
  for (SubcompactionState& sub_compact : compact_->sub_compact_states) {
1756
    const auto& sub_status = sub_compact.status;
I
Igor Canadi 已提交
1757

1758 1759 1760 1761 1762 1763 1764
    if (sub_compact.builder != nullptr) {
      // May happen if we get a shutdown call in the middle of compaction
      sub_compact.builder->Abandon();
      sub_compact.builder.reset();
    } else {
      assert(!sub_status.ok() || sub_compact.outfile == nullptr);
    }
1765
    for (const auto& out : sub_compact.outputs) {
1766 1767 1768
      // If this file was inserted into the table cache then remove
      // them here because this compaction was not committed.
      if (!sub_status.ok()) {
1769
        TableCache::Evict(table_cache_.get(), out.meta.fd.GetNumber());
1770
      }
I
Igor Canadi 已提交
1771
    }
1772 1773 1774
    // TODO: sub_compact.io_status is not checked like status. Not sure if thats
    // intentional. So ignoring the io_status as of now.
    sub_compact.io_status.PermitUncheckedError();
I
Igor Canadi 已提交
1775 1776 1777 1778 1779
  }
  delete compact_;
  compact_ = nullptr;
}

1780 1781
#ifndef ROCKSDB_LITE
namespace {
1782
void CopyPrefix(const Slice& src, size_t prefix_length, std::string* dst) {
1783 1784 1785
  assert(prefix_length > 0);
  size_t length = src.size() > prefix_length ? prefix_length : src.size();
  dst->assign(src.data(), length);
1786 1787 1788 1789 1790
}
}  // namespace

#endif  // !ROCKSDB_LITE

A
Andres Notzli 已提交
1791
void CompactionJob::UpdateCompactionStats() {
1792 1793
  assert(compact_);

1794 1795 1796 1797 1798 1799
  Compaction* compaction = compact_->compaction;
  compaction_stats_.num_input_files_in_non_output_levels = 0;
  compaction_stats_.num_input_files_in_output_level = 0;
  for (int input_level = 0;
       input_level < static_cast<int>(compaction->num_input_levels());
       ++input_level) {
1800
    if (compaction->level(input_level) != compaction->output_level()) {
1801 1802
      UpdateCompactionInputStatsHelper(
          &compaction_stats_.num_input_files_in_non_output_levels,
1803
          &compaction_stats_.bytes_read_non_output_levels, input_level);
1804 1805 1806
    } else {
      UpdateCompactionInputStatsHelper(
          &compaction_stats_.num_input_files_in_output_level,
1807
          &compaction_stats_.bytes_read_output_level, input_level);
1808 1809
    }
  }
A
Andres Notzli 已提交
1810

1811 1812 1813 1814 1815
  compaction_stats_.num_output_files =
      static_cast<int>(compact_->num_output_files) +
      static_cast<int>(compact_->num_blob_output_files);
  compaction_stats_.bytes_written =
      compact_->total_bytes + compact_->total_blob_bytes;
1816

1817
  if (compaction_stats_.num_input_records > compact_->num_output_records) {
1818
    compaction_stats_.num_dropped_records =
1819
        compaction_stats_.num_input_records - compact_->num_output_records;
A
Andres Notzli 已提交
1820
  }
1821 1822
}

1823 1824 1825
void CompactionJob::UpdateCompactionInputStatsHelper(int* num_files,
                                                     uint64_t* bytes_read,
                                                     int input_level) {
1826 1827 1828 1829 1830 1831 1832 1833 1834 1835 1836 1837
  const Compaction* compaction = compact_->compaction;
  auto num_input_files = compaction->num_input_files(input_level);
  *num_files += static_cast<int>(num_input_files);

  for (size_t i = 0; i < num_input_files; ++i) {
    const auto* file_meta = compaction->input(input_level, i);
    *bytes_read += file_meta->fd.GetFileSize();
    compaction_stats_.num_input_records +=
        static_cast<uint64_t>(file_meta->num_entries);
  }
}

1838 1839 1840
void CompactionJob::UpdateCompactionJobStats(
    const InternalStats::CompactionStats& stats) const {
#ifndef ROCKSDB_LITE
1841 1842 1843 1844 1845 1846 1847 1848 1849 1850 1851 1852 1853 1854 1855 1856 1857
  compaction_job_stats_->elapsed_micros = stats.micros;

  // input information
  compaction_job_stats_->total_input_bytes =
      stats.bytes_read_non_output_levels + stats.bytes_read_output_level;
  compaction_job_stats_->num_input_records = stats.num_input_records;
  compaction_job_stats_->num_input_files =
      stats.num_input_files_in_non_output_levels +
      stats.num_input_files_in_output_level;
  compaction_job_stats_->num_input_files_at_output_level =
      stats.num_input_files_in_output_level;

  // output information
  compaction_job_stats_->total_output_bytes = stats.bytes_written;
  compaction_job_stats_->num_output_records = compact_->num_output_records;
  compaction_job_stats_->num_output_files = stats.num_output_files;

1858
  if (stats.num_output_files > 0) {
1859 1860 1861 1862 1863
    CopyPrefix(compact_->SmallestUserKey(),
               CompactionJobStats::kMaxPrefixLength,
               &compaction_job_stats_->smallest_output_key_prefix);
    CopyPrefix(compact_->LargestUserKey(), CompactionJobStats::kMaxPrefixLength,
               &compaction_job_stats_->largest_output_key_prefix);
1864
  }
1865 1866
#else
  (void)stats;
1867 1868 1869
#endif  // !ROCKSDB_LITE
}

1870 1871 1872 1873
void CompactionJob::LogCompaction() {
  Compaction* compaction = compact_->compaction;
  ColumnFamilyData* cfd = compaction->column_family_data();

A
Andres Notzli 已提交
1874 1875 1876 1877
  // Let's check if anything will get logged. Don't prepare all the info if
  // we're not logging
  if (db_options_.info_log_level <= InfoLogLevel::INFO_LEVEL) {
    Compaction::InputLevelSummaryBuffer inputs_summary;
1878 1879 1880 1881
    ROCKS_LOG_INFO(
        db_options_.info_log, "[%s] [JOB %d] Compacting %s, score %.2f",
        cfd->GetName().c_str(), job_id_,
        compaction->InputLevelSummary(&inputs_summary), compaction->score());
A
Andres Notzli 已提交
1882 1883
    char scratch[2345];
    compaction->Summary(scratch, sizeof(scratch));
1884 1885
    ROCKS_LOG_INFO(db_options_.info_log, "[%s] Compaction start summary: %s\n",
                   cfd->GetName().c_str(), scratch);
A
Andres Notzli 已提交
1886 1887
    // build event logger report
    auto stream = event_logger_->Log();
1888 1889 1890 1891
    stream << "job" << job_id_ << "event"
           << "compaction_started"
           << "compaction_reason"
           << GetCompactionReasonString(compaction->compaction_reason());
A
Andres Notzli 已提交
1892 1893 1894 1895 1896 1897 1898 1899 1900 1901 1902 1903 1904
    for (size_t i = 0; i < compaction->num_input_levels(); ++i) {
      stream << ("files_L" + ToString(compaction->level(i)));
      stream.StartArray();
      for (auto f : *compaction->inputs(i)) {
        stream << f->fd.GetNumber();
      }
      stream.EndArray();
    }
    stream << "score" << compaction->score() << "input_data_size"
           << compaction->CalculateTotalInputSize();
  }
}

1905
}  // namespace ROCKSDB_NAMESPACE