compaction_job_test.cc 46.0 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
#ifndef ROCKSDB_LITE

A
Andres Notzli 已提交
8
#include <algorithm>
9
#include <array>
10
#include <cinttypes>
I
Igor Canadi 已提交
11 12
#include <map>
#include <string>
A
Andres Notzli 已提交
13
#include <tuple>
I
Igor Canadi 已提交
14

15
#include "db/blob/blob_index.h"
I
Igor Canadi 已提交
16
#include "db/column_family.h"
17
#include "db/compaction/compaction_job.h"
18
#include "db/db_impl/db_impl.h"
19
#include "db/error_handler.h"
I
Igor Canadi 已提交
20
#include "db/version_set.h"
21
#include "file/writable_file_writer.h"
I
Igor Canadi 已提交
22 23
#include "rocksdb/cache.h"
#include "rocksdb/db.h"
A
Andres Notzli 已提交
24
#include "rocksdb/options.h"
25
#include "rocksdb/write_buffer_manager.h"
A
Andres Notzli 已提交
26
#include "table/mock_table.h"
27 28
#include "test_util/testharness.h"
#include "test_util/testutil.h"
29
#include "util/string_util.h"
A
Andres Notzli 已提交
30
#include "utilities/merge_operators.h"
I
Igor Canadi 已提交
31

32
namespace ROCKSDB_NAMESPACE {
I
Igor Canadi 已提交
33

34
namespace {
A
Andres Notzli 已提交
35

36 37 38 39 40 41 42 43 44 45 46 47
void VerifyInitializationOfCompactionJobStats(
      const CompactionJobStats& compaction_job_stats) {
#if !defined(IOS_CROSS_COMPILE)
  ASSERT_EQ(compaction_job_stats.elapsed_micros, 0U);

  ASSERT_EQ(compaction_job_stats.num_input_records, 0U);
  ASSERT_EQ(compaction_job_stats.num_input_files, 0U);
  ASSERT_EQ(compaction_job_stats.num_input_files_at_output_level, 0U);

  ASSERT_EQ(compaction_job_stats.num_output_records, 0U);
  ASSERT_EQ(compaction_job_stats.num_output_files, 0U);

48
  ASSERT_EQ(compaction_job_stats.is_manual_compaction, true);
49 50 51 52 53 54 55 56 57 58 59

  ASSERT_EQ(compaction_job_stats.total_input_bytes, 0U);
  ASSERT_EQ(compaction_job_stats.total_output_bytes, 0U);

  ASSERT_EQ(compaction_job_stats.total_input_raw_key_bytes, 0U);
  ASSERT_EQ(compaction_job_stats.total_input_raw_value_bytes, 0U);

  ASSERT_EQ(compaction_job_stats.smallest_output_key_prefix[0], 0);
  ASSERT_EQ(compaction_job_stats.largest_output_key_prefix[0], 0);

  ASSERT_EQ(compaction_job_stats.num_records_replaced, 0U);
60 61 62 63 64

  ASSERT_EQ(compaction_job_stats.num_input_deletion_records, 0U);
  ASSERT_EQ(compaction_job_stats.num_expired_deletion_records, 0U);

  ASSERT_EQ(compaction_job_stats.num_corrupt_keys, 0U);
65 66 67 68 69
#endif  // !defined(IOS_CROSS_COMPILE)
}

}  // namespace

70 71 72 73
class CompactionJobTestBase : public testing::Test {
 protected:
  CompactionJobTestBase(std::string dbname, const Comparator* ucmp,
                        std::function<std::string(uint64_t)> encode_u64_ts)
I
Igor Canadi 已提交
74
      : env_(Env::Default()),
75
        fs_(std::make_shared<LegacyFileSystemWrapper>(env_)),
76 77
        dbname_(std::move(dbname)),
        ucmp_(ucmp),
78 79
        db_options_(),
        mutable_cf_options_(cf_options_),
80
        mutable_db_options_(),
81
        table_cache_(NewLRUCache(50000, 16)),
82
        write_buffer_manager_(db_options_.db_write_buffer_size),
83 84 85 86
        versions_(new VersionSet(
            dbname_, &db_options_, env_options_, table_cache_.get(),
            &write_buffer_manager_, &write_controller_,
            /*block_cache_tracer=*/nullptr, /*io_tracer=*/nullptr)),
I
Igor Canadi 已提交
87
        shutting_down_(false),
88
        preserve_deletes_seqnum_(0),
89
        mock_table_factory_(new mock::MockTableFactory()),
90 91 92 93
        error_handler_(nullptr, db_options_, &mutex_),
        encode_u64_ts_(std::move(encode_u64_ts)) {}

  void SetUp() override {
94
    EXPECT_OK(env_->CreateDirIfMissing(dbname_));
95 96
    db_options_.env = env_;
    db_options_.fs = fs_;
I
Igor Canadi 已提交
97 98
    db_options_.db_paths.emplace_back(dbname_,
                                      std::numeric_limits<uint64_t>::max());
99 100
    cf_options_.comparator = ucmp_;
    cf_options_.table_factory = mock_table_factory_;
I
Igor Canadi 已提交
101 102 103 104 105 106 107 108 109 110
  }

  std::string GenerateFileName(uint64_t file_number) {
    FileMetaData meta;
    std::vector<DbPath> db_paths;
    db_paths.emplace_back(dbname_, std::numeric_limits<uint64_t>::max());
    meta.fd = FileDescriptor(file_number, 0, 0);
    return TableFileName(db_paths, meta.fd.GetNumber(), meta.fd.GetPathId());
  }

111 112 113 114
  std::string KeyStr(const std::string& user_key, const SequenceNumber seq_num,
                     const ValueType t, uint64_t ts = 0) {
    std::string user_key_with_ts = user_key + encode_u64_ts_(ts);
    return InternalKey(user_key_with_ts, seq_num, t).Encode().ToString();
A
Andres Notzli 已提交
115 116
  }

117 118 119 120 121 122 123 124 125 126 127 128 129 130 131 132 133 134 135 136 137 138 139
  static std::string BlobStr(uint64_t blob_file_number, uint64_t offset,
                             uint64_t size) {
    std::string blob_index;
    BlobIndex::EncodeBlob(&blob_index, blob_file_number, offset, size,
                          kNoCompression);
    return blob_index;
  }

  static std::string BlobStrTTL(uint64_t blob_file_number, uint64_t offset,
                                uint64_t size, uint64_t expiration) {
    std::string blob_index;
    BlobIndex::EncodeBlobTTL(&blob_index, expiration, blob_file_number, offset,
                             size, kNoCompression);
    return blob_index;
  }

  static std::string BlobStrInlinedTTL(const Slice& value,
                                       uint64_t expiration) {
    std::string blob_index;
    BlobIndex::EncodeInlinedTTL(&blob_index, expiration, value);
    return blob_index;
  }

140
  void AddMockFile(const mock::KVVector& contents, int level = 0) {
A
Andres Notzli 已提交
141 142 143 144 145 146 147
    assert(contents.size() > 0);

    bool first_key = true;
    std::string smallest, largest;
    InternalKey smallest_key, largest_key;
    SequenceNumber smallest_seqno = kMaxSequenceNumber;
    SequenceNumber largest_seqno = 0;
148
    uint64_t oldest_blob_file_number = kInvalidBlobFileNumber;
A
Andres Notzli 已提交
149 150 151 152 153
    for (auto kv : contents) {
      ParsedInternalKey key;
      std::string skey;
      std::string value;
      std::tie(skey, value) = kv;
154 155
      const Status pik_status =
          ParseInternalKey(skey, &key, true /* log_err_key */);
A
Andres Notzli 已提交
156 157 158 159 160 161 162 163 164 165 166 167 168 169 170 171

      smallest_seqno = std::min(smallest_seqno, key.sequence);
      largest_seqno = std::max(largest_seqno, key.sequence);

      if (first_key ||
          cfd_->user_comparator()->Compare(key.user_key, smallest) < 0) {
        smallest.assign(key.user_key.data(), key.user_key.size());
        smallest_key.DecodeFrom(skey);
      }
      if (first_key ||
          cfd_->user_comparator()->Compare(key.user_key, largest) > 0) {
        largest.assign(key.user_key.data(), key.user_key.size());
        largest_key.DecodeFrom(skey);
      }

      first_key = false;
172

173
      if (pik_status.ok() && key.type == kTypeBlobIndex) {
174 175 176 177 178 179 180 181 182 183 184 185 186 187 188 189
        BlobIndex blob_index;
        const Status s = blob_index.DecodeFrom(value);
        if (!s.ok()) {
          continue;
        }

        if (blob_index.IsInlined() || blob_index.HasTTL() ||
            blob_index.file_number() == kInvalidBlobFileNumber) {
          continue;
        }

        if (oldest_blob_file_number == kInvalidBlobFileNumber ||
            oldest_blob_file_number > blob_index.file_number()) {
          oldest_blob_file_number = blob_index.file_number();
        }
      }
A
Andres Notzli 已提交
190 191 192 193 194 195 196 197
    }

    uint64_t file_number = versions_->NewFileNumber();
    EXPECT_OK(mock_table_factory_->CreateMockTable(
        env_, GenerateFileName(file_number), std::move(contents)));

    VersionEdit edit;
    edit.AddFile(level, file_number, 0, 10, smallest_key, largest_key,
198
                 smallest_seqno, largest_seqno, false, oldest_blob_file_number,
199 200
                 kUnknownOldestAncesterTime, kUnknownFileCreationTime,
                 kUnknownFileChecksum, kUnknownFileChecksumFuncName);
A
Andres Notzli 已提交
201 202

    mutex_.Lock();
203 204 205
    EXPECT_OK(
        versions_->LogAndApply(versions_->GetColumnFamilySet()->GetDefault(),
                               mutable_cf_options_, &edit, &mutex_));
A
Andres Notzli 已提交
206 207 208 209
    mutex_.Unlock();
  }

  void SetLastSequence(const SequenceNumber sequence_number) {
210
    versions_->SetLastAllocatedSequence(sequence_number + 1);
211
    versions_->SetLastPublishedSequence(sequence_number + 1);
A
Andres Notzli 已提交
212 213 214
    versions_->SetLastSequence(sequence_number + 1);
  }

I
Igor Canadi 已提交
215
  // returns expected result after compaction
216 217
  mock::KVVector CreateTwoFiles(bool gen_corrupted_keys) {
    stl_wrappers::KVMap expected_results;
218 219 220
    constexpr int kKeysPerFile = 10000;
    constexpr int kCorruptKeysPerFile = 200;
    constexpr int kMatchingKeys = kKeysPerFile / 2;
I
Igor Canadi 已提交
221
    SequenceNumber sequence_number = 0;
222 223 224 225 226

    auto corrupt_id = [&](int id) {
      return gen_corrupted_keys && id > 0 && id <= kCorruptKeysPerFile;
    };

I
Igor Canadi 已提交
227
    for (int i = 0; i < 2; ++i) {
228
      auto contents = mock::MakeMockFile();
I
Igor Canadi 已提交
229
      for (int k = 0; k < kKeysPerFile; ++k) {
230
        auto key = ToString(i * kMatchingKeys + k);
231
        auto value = ToString(i * kKeysPerFile + k);
I
Igor Canadi 已提交
232
        InternalKey internal_key(key, ++sequence_number, kTypeValue);
233

234 235
        // This is how the key will look like once it's written in bottommost
        // file
236
        InternalKey bottommost_internal_key(
237
            key, 0, kTypeValue);
238

239
        if (corrupt_id(k)) {
240 241
          test::CorruptKeyType(&internal_key);
          test::CorruptKeyType(&bottommost_internal_key);
242
        }
243
        contents.push_back({internal_key.Encode().ToString(), value});
244
        if (i == 1 || k < kMatchingKeys || corrupt_id(k - kMatchingKeys)) {
I
Igor Canadi 已提交
245
          expected_results.insert(
246
              {bottommost_internal_key.Encode().ToString(), value});
I
Igor Canadi 已提交
247 248
        }
      }
249
      mock::SortKVVector(&contents, ucmp_);
I
Igor Canadi 已提交
250

A
Andres Notzli 已提交
251 252
      AddMockFile(contents);
    }
I
Igor Canadi 已提交
253

A
Andres Notzli 已提交
254
    SetLastSequence(sequence_number);
I
Igor Canadi 已提交
255

256 257 258 259 260 261
    mock::KVVector expected_results_kvvector;
    for (auto& kv : expected_results) {
      expected_results_kvvector.push_back({kv.first, kv.second});
    }

    return expected_results_kvvector;
I
Igor Canadi 已提交
262 263
  }

I
Igor Canadi 已提交
264
  void NewDB() {
265
    EXPECT_OK(DestroyDB(dbname_, Options()));
266
    EXPECT_OK(env_->CreateDirIfMissing(dbname_));
267 268 269 270
    versions_.reset(
        new VersionSet(dbname_, &db_options_, env_options_, table_cache_.get(),
                       &write_buffer_manager_, &write_controller_,
                       /*block_cache_tracer=*/nullptr, /*io_tracer=*/nullptr));
271
    compaction_job_stats_.Reset();
272
    SetIdentityFile(env_, dbname_);
273

I
Igor Canadi 已提交
274 275 276 277 278 279
    VersionEdit new_db;
    new_db.SetLogNumber(0);
    new_db.SetNextFile(2);
    new_db.SetLastSequence(0);

    const std::string manifest = DescriptorFileName(dbname_, 1);
280
    std::unique_ptr<WritableFile> file;
I
Igor Canadi 已提交
281 282 283
    Status s = env_->NewWritableFile(
        manifest, &file, env_->OptimizeForManifestWrite(env_options_));
    ASSERT_OK(s);
284 285
    std::unique_ptr<WritableFileWriter> file_writer(new WritableFileWriter(
        NewLegacyWritableFileWrapper(std::move(file)), manifest, env_options_));
I
Igor Canadi 已提交
286
    {
287
      log::Writer log(std::move(file_writer), 0, false);
I
Igor Canadi 已提交
288 289 290 291 292 293
      std::string record;
      new_db.EncodeTo(&record);
      s = log.AddRecord(record);
    }
    ASSERT_OK(s);
    // Make "CURRENT" file that points to the new manifest file.
294
    s = SetCurrentFile(fs_.get(), dbname_, 1, nullptr);
A
Andres Notzli 已提交
295

296 297
    ASSERT_OK(s);

I
Igor Canadi 已提交
298 299
    cf_options_.merge_operator = merge_op_;
    cf_options_.compaction_filter = compaction_filter_.get();
300
    std::vector<ColumnFamilyDescriptor> column_families;
A
Andres Notzli 已提交
301 302
    column_families.emplace_back(kDefaultColumnFamilyName, cf_options_);

303
    ASSERT_OK(versions_->Recover(column_families, false));
A
Andres Notzli 已提交
304
    cfd_ = versions_->GetColumnFamilySet()->GetDefault();
I
Igor Canadi 已提交
305 306
  }

307 308
  void RunCompaction(
      const std::vector<std::vector<FileMetaData*>>& input_files,
309
      const mock::KVVector& expected_results,
310
      const std::vector<SequenceNumber>& snapshots = {},
311
      SequenceNumber earliest_write_conflict_snapshot = kMaxSequenceNumber,
312 313
      int output_level = 1, bool verify = true,
      uint64_t expected_oldest_blob_file_number = kInvalidBlobFileNumber) {
314 315
    auto cfd = versions_->GetColumnFamilySet()->GetDefault();

A
Andres Notzli 已提交
316 317 318 319 320
    size_t num_input_files = 0;
    std::vector<CompactionInputFiles> compaction_input_files;
    for (size_t level = 0; level < input_files.size(); level++) {
      auto level_files = input_files[level];
      CompactionInputFiles compaction_level;
I
Islam AbdelRahman 已提交
321
      compaction_level.level = static_cast<int>(level);
A
Andres Notzli 已提交
322 323 324 325
      compaction_level.files.insert(compaction_level.files.end(),
          level_files.begin(), level_files.end());
      compaction_input_files.push_back(compaction_level);
      num_input_files += level_files.size();
326
    }
A
Andres Notzli 已提交
327

328 329
    Compaction compaction(
        cfd->current()->storage_info(), *cfd->ioptions(),
330 331 332 333
        *cfd->GetLatestMutableCFOptions(), mutable_db_options_,
        compaction_input_files, output_level, 1024 * 1024, 10 * 1024 * 1024, 0,
        kNoCompression, cfd->GetLatestMutableCFOptions()->compression_opts, 0,
        {}, true);
334 335 336 337 338
    compaction.SetInputVersion(cfd->current());

    LogBuffer log_buffer(InfoLogLevel::INFO_LEVEL, db_options_.info_log.get());
    mutex_.Lock();
    EventLogger event_logger(db_options_.info_log.get());
Y
Yi Wu 已提交
339 340
    // TODO(yiwu) add a mock snapshot checker and add test for it.
    SnapshotChecker* snapshot_checker = nullptr;
341 342
    ASSERT_TRUE(full_history_ts_low_.empty() ||
                ucmp_->timestamp_size() == full_history_ts_low_.size());
343 344 345
    CompactionJob compaction_job(
        0, &compaction, db_options_, env_options_, versions_.get(),
        &shutting_down_, preserve_deletes_seqnum_, &log_buffer, nullptr,
346
        nullptr, nullptr, nullptr, &mutex_, &error_handler_, snapshots,
347
        earliest_write_conflict_snapshot, snapshot_checker, table_cache_,
348
        &event_logger, false, false, dbname_, &compaction_job_stats_,
349 350 351
        Env::Priority::USER, nullptr /* IOTracer */,
        /*manual_compaction_paused=*/nullptr, /*db_id=*/"",
        /*db_session_id=*/"", full_history_ts_low_);
352
    VerifyInitializationOfCompactionJobStats(compaction_job_stats_);
353 354 355

    compaction_job.Prepare();
    mutex_.Unlock();
356
    Status s = compaction_job.Run();
A
Andres Notzli 已提交
357
    ASSERT_OK(s);
358
    ASSERT_OK(compaction_job.io_status());
A
Andres Notzli 已提交
359
    mutex_.Lock();
360
    ASSERT_OK(compaction_job.Install(*cfd->GetLatestMutableCFOptions()));
361
    ASSERT_OK(compaction_job.io_status());
362 363
    mutex_.Unlock();

364
    if (verify) {
365 366 367 368
      ASSERT_GE(compaction_job_stats_.elapsed_micros, 0U);
      ASSERT_EQ(compaction_job_stats_.num_input_files, num_input_files);

      if (expected_results.empty()) {
369 370 371 372
        ASSERT_EQ(compaction_job_stats_.num_output_files, 0U);
      } else {
        ASSERT_EQ(compaction_job_stats_.num_output_files, 1U);
        mock_table_factory_->AssertLatestFile(expected_results);
373 374 375 376 377 378

        auto output_files =
            cfd->current()->storage_info()->LevelFiles(output_level);
        ASSERT_EQ(output_files.size(), 1);
        ASSERT_EQ(output_files[0]->oldest_blob_file_number,
                  expected_oldest_blob_file_number);
379
      }
I
Igor Canadi 已提交
380
    }
381 382
  }

I
Igor Canadi 已提交
383
  Env* env_;
384
  std::shared_ptr<FileSystem> fs_;
I
Igor Canadi 已提交
385
  std::string dbname_;
386
  const Comparator* const ucmp_;
I
Igor Canadi 已提交
387
  EnvOptions env_options_;
388 389
  ImmutableDBOptions db_options_;
  ColumnFamilyOptions cf_options_;
I
Igor Canadi 已提交
390
  MutableCFOptions mutable_cf_options_;
391
  MutableDBOptions mutable_db_options_;
I
Igor Canadi 已提交
392 393
  std::shared_ptr<Cache> table_cache_;
  WriteController write_controller_;
394
  WriteBufferManager write_buffer_manager_;
I
Igor Canadi 已提交
395
  std::unique_ptr<VersionSet> versions_;
396
  InstrumentedMutex mutex_;
I
Igor Canadi 已提交
397
  std::atomic<bool> shutting_down_;
398
  SequenceNumber preserve_deletes_seqnum_;
I
Igor Canadi 已提交
399
  std::shared_ptr<mock::MockTableFactory> mock_table_factory_;
400
  CompactionJobStats compaction_job_stats_;
A
Andres Notzli 已提交
401
  ColumnFamilyData* cfd_;
I
Igor Canadi 已提交
402 403
  std::unique_ptr<CompactionFilter> compaction_filter_;
  std::shared_ptr<MergeOperator> merge_op_;
404
  ErrorHandler error_handler_;
405 406 407 408 409 410 411 412 413 414 415
  std::string full_history_ts_low_;
  const std::function<std::string(uint64_t)> encode_u64_ts_;
};

// TODO(icanadi) Make it simpler once we mock out VersionSet
class CompactionJobTest : public CompactionJobTestBase {
 public:
  CompactionJobTest()
      : CompactionJobTestBase(test::PerThreadDBPath("compaction_job_test"),
                              BytewiseComparator(),
                              [](uint64_t /*ts*/) { return ""; }) {}
I
Igor Canadi 已提交
416 417
};

I
Igor Sugak 已提交
418
TEST_F(CompactionJobTest, Simple) {
A
Andres Notzli 已提交
419 420
  NewDB();

421
  auto expected_results = CreateTwoFiles(false);
I
Igor Canadi 已提交
422 423 424
  auto cfd = versions_->GetColumnFamilySet()->GetDefault();
  auto files = cfd->current()->storage_info()->LevelFiles(0);
  ASSERT_EQ(2U, files.size());
A
Andres Notzli 已提交
425
  RunCompaction({ files }, expected_results);
I
Igor Canadi 已提交
426 427
}

428
TEST_F(CompactionJobTest, DISABLED_SimpleCorrupted) {
A
Andres Notzli 已提交
429 430
  NewDB();

431 432 433
  auto expected_results = CreateTwoFiles(true);
  auto cfd = versions_->GetColumnFamilySet()->GetDefault();
  auto files = cfd->current()->storage_info()->LevelFiles(0);
I
Igor Canadi 已提交
434
  RunCompaction({files}, expected_results);
435
  ASSERT_EQ(compaction_job_stats_.num_corrupt_keys, 400U);
A
Andres Notzli 已提交
436 437 438 439 440
}

TEST_F(CompactionJobTest, SimpleDeletion) {
  NewDB();

441 442
  auto file1 = mock::MakeMockFile({{KeyStr("c", 4U, kTypeDeletion), ""},
                                   {KeyStr("c", 3U, kTypeValue), "val"}});
A
Andres Notzli 已提交
443 444
  AddMockFile(file1);

445 446
  auto file2 = mock::MakeMockFile({{KeyStr("b", 2U, kTypeValue), "val"},
                                   {KeyStr("b", 1U, kTypeValue), "val"}});
A
Andres Notzli 已提交
447 448
  AddMockFile(file2);

449 450
  auto expected_results =
      mock::MakeMockFile({{KeyStr("b", 0U, kTypeValue), "val"}});
A
Andres Notzli 已提交
451 452 453

  SetLastSequence(4U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
I
Igor Canadi 已提交
454
  RunCompaction({files}, expected_results);
A
Andres Notzli 已提交
455 456
}

Z
zhangjinpeng1987 已提交
457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474
TEST_F(CompactionJobTest, OutputNothing) {
  NewDB();

  auto file1 = mock::MakeMockFile({{KeyStr("a", 1U, kTypeValue), "val"}});

  AddMockFile(file1);

  auto file2 = mock::MakeMockFile({{KeyStr("a", 2U, kTypeDeletion), ""}});

  AddMockFile(file2);

  auto expected_results = mock::MakeMockFile();

  SetLastSequence(4U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results);
}

A
Andres Notzli 已提交
475 476 477
TEST_F(CompactionJobTest, SimpleOverwrite) {
  NewDB();

478 479 480 481
  auto file1 = mock::MakeMockFile({
      {KeyStr("a", 3U, kTypeValue), "val2"},
      {KeyStr("b", 4U, kTypeValue), "val3"},
  });
A
Andres Notzli 已提交
482 483
  AddMockFile(file1);

484 485
  auto file2 = mock::MakeMockFile({{KeyStr("a", 1U, kTypeValue), "val"},
                                   {KeyStr("b", 2U, kTypeValue), "val"}});
A
Andres Notzli 已提交
486 487
  AddMockFile(file2);

488 489
  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 0U, kTypeValue), "val2"},
490
                          {KeyStr("b", 0U, kTypeValue), "val3"}});
A
Andres Notzli 已提交
491 492 493

  SetLastSequence(4U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
I
Igor Canadi 已提交
494
  RunCompaction({files}, expected_results);
A
Andres Notzli 已提交
495 496 497 498 499
}

TEST_F(CompactionJobTest, SimpleNonLastLevel) {
  NewDB();

500 501 502 503
  auto file1 = mock::MakeMockFile({
      {KeyStr("a", 5U, kTypeValue), "val2"},
      {KeyStr("b", 6U, kTypeValue), "val3"},
  });
A
Andres Notzli 已提交
504 505
  AddMockFile(file1);

506 507
  auto file2 = mock::MakeMockFile({{KeyStr("a", 3U, kTypeValue), "val"},
                                   {KeyStr("b", 4U, kTypeValue), "val"}});
A
Andres Notzli 已提交
508 509
  AddMockFile(file2, 1);

510 511
  auto file3 = mock::MakeMockFile({{KeyStr("a", 1U, kTypeValue), "val"},
                                   {KeyStr("b", 2U, kTypeValue), "val"}});
A
Andres Notzli 已提交
512 513 514 515
  AddMockFile(file3, 2);

  // Because level 1 is not the last level, the sequence numbers of a and b
  // cannot be set to 0
516 517 518
  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 5U, kTypeValue), "val2"},
                          {KeyStr("b", 6U, kTypeValue), "val3"}});
A
Andres Notzli 已提交
519 520 521 522

  SetLastSequence(6U);
  auto lvl0_files = cfd_->current()->storage_info()->LevelFiles(0);
  auto lvl1_files = cfd_->current()->storage_info()->LevelFiles(1);
I
Igor Canadi 已提交
523
  RunCompaction({lvl0_files, lvl1_files}, expected_results);
A
Andres Notzli 已提交
524 525 526
}

TEST_F(CompactionJobTest, SimpleMerge) {
I
Igor Canadi 已提交
527 528
  merge_op_ = MergeOperators::CreateStringAppendOperator();
  NewDB();
A
Andres Notzli 已提交
529

530 531 532 533 534
  auto file1 = mock::MakeMockFile({
      {KeyStr("a", 5U, kTypeMerge), "5"},
      {KeyStr("a", 4U, kTypeMerge), "4"},
      {KeyStr("a", 3U, kTypeValue), "3"},
  });
A
Andres Notzli 已提交
535 536
  AddMockFile(file1);

537 538
  auto file2 = mock::MakeMockFile(
      {{KeyStr("b", 2U, kTypeMerge), "2"}, {KeyStr("b", 1U, kTypeValue), "1"}});
A
Andres Notzli 已提交
539 540
  AddMockFile(file2);

541 542
  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 0U, kTypeValue), "3,4,5"},
543
                          {KeyStr("b", 0U, kTypeValue), "1,2"}});
A
Andres Notzli 已提交
544 545 546

  SetLastSequence(5U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
I
Igor Canadi 已提交
547
  RunCompaction({files}, expected_results);
A
Andres Notzli 已提交
548 549 550
}

TEST_F(CompactionJobTest, NonAssocMerge) {
I
Igor Canadi 已提交
551 552
  merge_op_ = MergeOperators::CreateStringAppendTESTOperator();
  NewDB();
A
Andres Notzli 已提交
553

554 555 556 557 558
  auto file1 = mock::MakeMockFile({
      {KeyStr("a", 5U, kTypeMerge), "5"},
      {KeyStr("a", 4U, kTypeMerge), "4"},
      {KeyStr("a", 3U, kTypeMerge), "3"},
  });
A
Andres Notzli 已提交
559 560
  AddMockFile(file1);

561 562
  auto file2 = mock::MakeMockFile(
      {{KeyStr("b", 2U, kTypeMerge), "2"}, {KeyStr("b", 1U, kTypeMerge), "1"}});
A
Andres Notzli 已提交
563 564
  AddMockFile(file2);

565 566
  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 0U, kTypeValue), "3,4,5"},
567
                          {KeyStr("b", 0U, kTypeValue), "1,2"}});
A
Andres Notzli 已提交
568 569 570

  SetLastSequence(5U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
I
Igor Canadi 已提交
571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593
  RunCompaction({files}, expected_results);
}

// Filters merge operands with value 10.
TEST_F(CompactionJobTest, MergeOperandFilter) {
  merge_op_ = MergeOperators::CreateUInt64AddOperator();
  compaction_filter_.reset(new test::FilterNumber(10U));
  NewDB();

  auto file1 = mock::MakeMockFile(
      {{KeyStr("a", 5U, kTypeMerge), test::EncodeInt(5U)},
       {KeyStr("a", 4U, kTypeMerge), test::EncodeInt(10U)},  // Filtered
       {KeyStr("a", 3U, kTypeMerge), test::EncodeInt(3U)}});
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile({
      {KeyStr("b", 2U, kTypeMerge), test::EncodeInt(2U)},
      {KeyStr("b", 1U, kTypeMerge), test::EncodeInt(10U)}  // Filtered
  });
  AddMockFile(file2);

  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 0U, kTypeValue), test::EncodeInt(8U)},
594
                          {KeyStr("b", 0U, kTypeValue), test::EncodeInt(2U)}});
I
Igor Canadi 已提交
595 596 597 598 599 600 601 602 603 604 605 606 607 608 609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667

  SetLastSequence(5U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results);
}

TEST_F(CompactionJobTest, FilterSomeMergeOperands) {
  merge_op_ = MergeOperators::CreateUInt64AddOperator();
  compaction_filter_.reset(new test::FilterNumber(10U));
  NewDB();

  auto file1 = mock::MakeMockFile(
      {{KeyStr("a", 5U, kTypeMerge), test::EncodeInt(5U)},
       {KeyStr("a", 4U, kTypeMerge), test::EncodeInt(10U)},  // Filtered
       {KeyStr("a", 3U, kTypeValue), test::EncodeInt(5U)},
       {KeyStr("d", 8U, kTypeMerge), test::EncodeInt(10U)}});
  AddMockFile(file1);

  auto file2 =
      mock::MakeMockFile({{KeyStr("b", 2U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 1U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("c", 2U, kTypeMerge), test::EncodeInt(3U)},
                          {KeyStr("c", 1U, kTypeValue), test::EncodeInt(7U)},
                          {KeyStr("d", 1U, kTypeValue), test::EncodeInt(6U)}});
  AddMockFile(file2);

  auto file3 =
      mock::MakeMockFile({{KeyStr("a", 1U, kTypeMerge), test::EncodeInt(3U)}});
  AddMockFile(file3, 2);

  auto expected_results = mock::MakeMockFile({
      {KeyStr("a", 5U, kTypeValue), test::EncodeInt(10U)},
      {KeyStr("c", 2U, kTypeValue), test::EncodeInt(10U)},
      {KeyStr("d", 1U, kTypeValue), test::EncodeInt(6U)}
      // b does not appear because the operands are filtered
  });

  SetLastSequence(5U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results);
}

// Test where all operands/merge results are filtered out.
TEST_F(CompactionJobTest, FilterAllMergeOperands) {
  merge_op_ = MergeOperators::CreateUInt64AddOperator();
  compaction_filter_.reset(new test::FilterNumber(10U));
  NewDB();

  auto file1 =
      mock::MakeMockFile({{KeyStr("a", 11U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("a", 10U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("a", 9U, kTypeMerge), test::EncodeInt(10U)}});
  AddMockFile(file1);

  auto file2 =
      mock::MakeMockFile({{KeyStr("b", 8U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 7U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 6U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 5U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 4U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 3U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 2U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("c", 2U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("c", 1U, kTypeMerge), test::EncodeInt(10U)}});
  AddMockFile(file2);

  auto file3 =
      mock::MakeMockFile({{KeyStr("a", 2U, kTypeMerge), test::EncodeInt(10U)},
                          {KeyStr("b", 1U, kTypeMerge), test::EncodeInt(10U)}});
  AddMockFile(file3, 2);

  SetLastSequence(11U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
I
Igor Canadi 已提交
668

669
  mock::KVVector empty_map;
I
Igor Canadi 已提交
670
  RunCompaction({files}, empty_map);
671 672
}

A
Andres Noetzli 已提交
673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689 690 691 692 693 694 695 696 697 698 699 700 701
TEST_F(CompactionJobTest, SimpleSingleDelete) {
  NewDB();

  auto file1 = mock::MakeMockFile({
      {KeyStr("a", 5U, kTypeDeletion), ""},
      {KeyStr("b", 6U, kTypeSingleDeletion), ""},
  });
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile({{KeyStr("a", 3U, kTypeValue), "val"},
                                   {KeyStr("b", 4U, kTypeValue), "val"}});
  AddMockFile(file2);

  auto file3 = mock::MakeMockFile({
      {KeyStr("a", 1U, kTypeValue), "val"},
  });
  AddMockFile(file3, 2);

  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 5U, kTypeDeletion), ""}});

  SetLastSequence(6U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results);
}

TEST_F(CompactionJobTest, SingleDeleteSnapshots) {
  NewDB();

702 703 704 705 706 707 708 709 710 711 712 713 714 715
  auto file1 = mock::MakeMockFile({
      {KeyStr("A", 12U, kTypeSingleDeletion), ""},
      {KeyStr("a", 12U, kTypeSingleDeletion), ""},
      {KeyStr("b", 21U, kTypeSingleDeletion), ""},
      {KeyStr("c", 22U, kTypeSingleDeletion), ""},
      {KeyStr("d", 9U, kTypeSingleDeletion), ""},
      {KeyStr("f", 21U, kTypeSingleDeletion), ""},
      {KeyStr("j", 11U, kTypeSingleDeletion), ""},
      {KeyStr("j", 9U, kTypeSingleDeletion), ""},
      {KeyStr("k", 12U, kTypeSingleDeletion), ""},
      {KeyStr("k", 11U, kTypeSingleDeletion), ""},
      {KeyStr("l", 3U, kTypeSingleDeletion), ""},
      {KeyStr("l", 2U, kTypeSingleDeletion), ""},
  });
A
Andres Noetzli 已提交
716 717
  AddMockFile(file1);

718 719 720 721 722 723 724 725 726 727 728 729 730 731
  auto file2 = mock::MakeMockFile({
      {KeyStr("0", 2U, kTypeSingleDeletion), ""},
      {KeyStr("a", 11U, kTypeValue), "val1"},
      {KeyStr("b", 11U, kTypeValue), "val2"},
      {KeyStr("c", 21U, kTypeValue), "val3"},
      {KeyStr("d", 8U, kTypeValue), "val4"},
      {KeyStr("e", 2U, kTypeSingleDeletion), ""},
      {KeyStr("f", 1U, kTypeValue), "val1"},
      {KeyStr("g", 11U, kTypeSingleDeletion), ""},
      {KeyStr("h", 2U, kTypeSingleDeletion), ""},
      {KeyStr("m", 12U, kTypeValue), "val1"},
      {KeyStr("m", 11U, kTypeSingleDeletion), ""},
      {KeyStr("m", 8U, kTypeValue), "val2"},
  });
A
Andres Noetzli 已提交
732 733
  AddMockFile(file2);

734 735 736 737
  auto file3 = mock::MakeMockFile({
      {KeyStr("A", 1U, kTypeValue), "val"},
      {KeyStr("e", 1U, kTypeValue), "val"},
  });
A
Andres Noetzli 已提交
738 739
  AddMockFile(file3, 2);

740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756 757
  auto expected_results = mock::MakeMockFile({
      {KeyStr("A", 12U, kTypeSingleDeletion), ""},
      {KeyStr("a", 12U, kTypeSingleDeletion), ""},
      {KeyStr("a", 11U, kTypeValue), ""},
      {KeyStr("b", 21U, kTypeSingleDeletion), ""},
      {KeyStr("b", 11U, kTypeValue), "val2"},
      {KeyStr("c", 22U, kTypeSingleDeletion), ""},
      {KeyStr("c", 21U, kTypeValue), ""},
      {KeyStr("e", 2U, kTypeSingleDeletion), ""},
      {KeyStr("f", 21U, kTypeSingleDeletion), ""},
      {KeyStr("f", 1U, kTypeValue), "val1"},
      {KeyStr("g", 11U, kTypeSingleDeletion), ""},
      {KeyStr("j", 11U, kTypeSingleDeletion), ""},
      {KeyStr("k", 11U, kTypeSingleDeletion), ""},
      {KeyStr("m", 12U, kTypeValue), "val1"},
      {KeyStr("m", 11U, kTypeSingleDeletion), ""},
      {KeyStr("m", 8U, kTypeValue), "val2"},
  });
A
Andres Noetzli 已提交
758 759 760

  SetLastSequence(22U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
761 762 763 764 765 766 767 768 769 770 771 772 773 774 775 776 777 778 779 780 781 782 783 784 785 786 787 788 789 790 791 792 793 794 795 796 797 798 799 800 801 802 803 804 805 806 807 808 809 810 811 812 813 814 815 816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839
  RunCompaction({files}, expected_results, {10U, 20U}, 10U);
}

TEST_F(CompactionJobTest, EarliestWriteConflictSnapshot) {
  NewDB();

  // Test multiple snapshots where the earliest snapshot is not a
  // write-conflic-snapshot.

  auto file1 = mock::MakeMockFile({
      {KeyStr("A", 24U, kTypeSingleDeletion), ""},
      {KeyStr("A", 23U, kTypeValue), "val"},
      {KeyStr("B", 24U, kTypeSingleDeletion), ""},
      {KeyStr("B", 23U, kTypeValue), "val"},
      {KeyStr("D", 24U, kTypeSingleDeletion), ""},
      {KeyStr("G", 32U, kTypeSingleDeletion), ""},
      {KeyStr("G", 31U, kTypeValue), "val"},
      {KeyStr("G", 24U, kTypeSingleDeletion), ""},
      {KeyStr("G", 23U, kTypeValue), "val2"},
      {KeyStr("H", 31U, kTypeValue), "val"},
      {KeyStr("H", 24U, kTypeSingleDeletion), ""},
      {KeyStr("H", 23U, kTypeValue), "val"},
      {KeyStr("I", 35U, kTypeSingleDeletion), ""},
      {KeyStr("I", 34U, kTypeValue), "val2"},
      {KeyStr("I", 33U, kTypeSingleDeletion), ""},
      {KeyStr("I", 32U, kTypeValue), "val3"},
      {KeyStr("I", 31U, kTypeSingleDeletion), ""},
      {KeyStr("J", 34U, kTypeValue), "val"},
      {KeyStr("J", 33U, kTypeSingleDeletion), ""},
      {KeyStr("J", 25U, kTypeValue), "val2"},
      {KeyStr("J", 24U, kTypeSingleDeletion), ""},
  });
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile({
      {KeyStr("A", 14U, kTypeSingleDeletion), ""},
      {KeyStr("A", 13U, kTypeValue), "val2"},
      {KeyStr("C", 14U, kTypeSingleDeletion), ""},
      {KeyStr("C", 13U, kTypeValue), "val"},
      {KeyStr("E", 12U, kTypeSingleDeletion), ""},
      {KeyStr("F", 4U, kTypeSingleDeletion), ""},
      {KeyStr("F", 3U, kTypeValue), "val"},
      {KeyStr("G", 14U, kTypeSingleDeletion), ""},
      {KeyStr("G", 13U, kTypeValue), "val3"},
      {KeyStr("H", 14U, kTypeSingleDeletion), ""},
      {KeyStr("H", 13U, kTypeValue), "val2"},
      {KeyStr("I", 13U, kTypeValue), "val4"},
      {KeyStr("I", 12U, kTypeSingleDeletion), ""},
      {KeyStr("I", 11U, kTypeValue), "val5"},
      {KeyStr("J", 15U, kTypeValue), "val3"},
      {KeyStr("J", 14U, kTypeSingleDeletion), ""},
  });
  AddMockFile(file2);

  auto expected_results = mock::MakeMockFile({
      {KeyStr("A", 24U, kTypeSingleDeletion), ""},
      {KeyStr("A", 23U, kTypeValue), ""},
      {KeyStr("B", 24U, kTypeSingleDeletion), ""},
      {KeyStr("B", 23U, kTypeValue), ""},
      {KeyStr("D", 24U, kTypeSingleDeletion), ""},
      {KeyStr("E", 12U, kTypeSingleDeletion), ""},
      {KeyStr("G", 32U, kTypeSingleDeletion), ""},
      {KeyStr("G", 31U, kTypeValue), ""},
      {KeyStr("H", 31U, kTypeValue), "val"},
      {KeyStr("I", 35U, kTypeSingleDeletion), ""},
      {KeyStr("I", 34U, kTypeValue), ""},
      {KeyStr("I", 31U, kTypeSingleDeletion), ""},
      {KeyStr("I", 13U, kTypeValue), "val4"},
      {KeyStr("J", 34U, kTypeValue), "val"},
      {KeyStr("J", 33U, kTypeSingleDeletion), ""},
      {KeyStr("J", 25U, kTypeValue), "val2"},
      {KeyStr("J", 24U, kTypeSingleDeletion), ""},
      {KeyStr("J", 15U, kTypeValue), "val3"},
      {KeyStr("J", 14U, kTypeSingleDeletion), ""},
  });

  SetLastSequence(24U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results, {10U, 20U, 30U}, 20U);
A
Andres Noetzli 已提交
840 841 842 843 844 845 846 847 848 849 850 851 852 853 854 855 856
}

TEST_F(CompactionJobTest, SingleDeleteZeroSeq) {
  NewDB();

  auto file1 = mock::MakeMockFile({
      {KeyStr("A", 10U, kTypeSingleDeletion), ""},
      {KeyStr("dummy", 5U, kTypeValue), "val2"},
  });
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile({
      {KeyStr("A", 0U, kTypeValue), "val"},
  });
  AddMockFile(file2);

  auto expected_results = mock::MakeMockFile({
857
      {KeyStr("dummy", 0U, kTypeValue), "val2"},
A
Andres Noetzli 已提交
858 859 860 861 862 863 864 865 866 867 868
  });

  SetLastSequence(22U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results, {});
}

TEST_F(CompactionJobTest, MultiSingleDelete) {
  // Tests three scenarios involving multiple single delete/put pairs:
  //
  // A: Put Snapshot SDel Put SDel -> Put Snapshot SDel
869
  // B: Snapshot Put SDel Put SDel Snapshot -> Snapshot SDel Snapshot
A
Andres Noetzli 已提交
870
  // C: SDel Put SDel Snapshot Put -> Snapshot Put
871 872 873 874 875 876 877 878 879 880 881 882 883 884
  // D: (Put) SDel Snapshot Put SDel -> (Put) SDel Snapshot SDel
  // E: Put SDel Snapshot Put SDel -> Snapshot SDel
  // F: Put SDel Put Sdel Snapshot -> removed
  // G: Snapshot SDel Put SDel Put -> Snapshot Put SDel
  // H: (Put) Put SDel Put Sdel Snapshot -> Removed
  // I: (Put) Snapshot Put SDel Put SDel -> SDel
  // J: Put Put SDel Put SDel SDel Snapshot Put Put SDel SDel Put
  //      -> Snapshot Put
  // K: SDel SDel Put SDel Put Put Snapshot SDel Put SDel SDel Put SDel
  //      -> Snapshot Put Snapshot SDel
  // L: SDel Put Del Put SDel Snapshot Del Put Del SDel Put SDel
  //      -> Snapshot SDel
  // M: (Put) SDel Put Del Put SDel Snapshot Put Del SDel Put SDel Del
  //      -> SDel Snapshot Del
A
Andres Noetzli 已提交
885 886 887 888 889 890 891 892 893 894 895
  NewDB();

  auto file1 = mock::MakeMockFile({
      {KeyStr("A", 14U, kTypeSingleDeletion), ""},
      {KeyStr("A", 13U, kTypeValue), "val5"},
      {KeyStr("A", 12U, kTypeSingleDeletion), ""},
      {KeyStr("B", 14U, kTypeSingleDeletion), ""},
      {KeyStr("B", 13U, kTypeValue), "val2"},
      {KeyStr("C", 14U, kTypeValue), "val3"},
      {KeyStr("D", 12U, kTypeSingleDeletion), ""},
      {KeyStr("D", 11U, kTypeValue), "val4"},
896 897 898 899 900 901 902 903 904 905 906 907 908 909 910 911 912 913 914 915 916 917 918 919 920 921 922 923
      {KeyStr("G", 15U, kTypeValue), "val"},
      {KeyStr("G", 14U, kTypeSingleDeletion), ""},
      {KeyStr("G", 13U, kTypeValue), "val"},
      {KeyStr("I", 14U, kTypeSingleDeletion), ""},
      {KeyStr("I", 13U, kTypeValue), "val"},
      {KeyStr("J", 15U, kTypeValue), "val"},
      {KeyStr("J", 14U, kTypeSingleDeletion), ""},
      {KeyStr("J", 13U, kTypeSingleDeletion), ""},
      {KeyStr("J", 12U, kTypeValue), "val"},
      {KeyStr("J", 11U, kTypeValue), "val"},
      {KeyStr("K", 16U, kTypeSingleDeletion), ""},
      {KeyStr("K", 15U, kTypeValue), "val1"},
      {KeyStr("K", 14U, kTypeSingleDeletion), ""},
      {KeyStr("K", 13U, kTypeSingleDeletion), ""},
      {KeyStr("K", 12U, kTypeValue), "val2"},
      {KeyStr("K", 11U, kTypeSingleDeletion), ""},
      {KeyStr("L", 16U, kTypeSingleDeletion), ""},
      {KeyStr("L", 15U, kTypeValue), "val"},
      {KeyStr("L", 14U, kTypeSingleDeletion), ""},
      {KeyStr("L", 13U, kTypeDeletion), ""},
      {KeyStr("L", 12U, kTypeValue), "val"},
      {KeyStr("L", 11U, kTypeDeletion), ""},
      {KeyStr("M", 16U, kTypeDeletion), ""},
      {KeyStr("M", 15U, kTypeSingleDeletion), ""},
      {KeyStr("M", 14U, kTypeValue), "val"},
      {KeyStr("M", 13U, kTypeSingleDeletion), ""},
      {KeyStr("M", 12U, kTypeDeletion), ""},
      {KeyStr("M", 11U, kTypeValue), "val"},
A
Andres Noetzli 已提交
924 925 926 927 928 929 930 931 932 933 934
  });
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile({
      {KeyStr("A", 10U, kTypeValue), "val"},
      {KeyStr("B", 12U, kTypeSingleDeletion), ""},
      {KeyStr("B", 11U, kTypeValue), "val2"},
      {KeyStr("C", 10U, kTypeSingleDeletion), ""},
      {KeyStr("C", 9U, kTypeValue), "val6"},
      {KeyStr("C", 8U, kTypeSingleDeletion), ""},
      {KeyStr("D", 10U, kTypeSingleDeletion), ""},
935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951 952 953 954 955 956 957 958 959 960 961 962 963 964 965 966 967 968 969 970
      {KeyStr("E", 12U, kTypeSingleDeletion), ""},
      {KeyStr("E", 11U, kTypeValue), "val"},
      {KeyStr("E", 5U, kTypeSingleDeletion), ""},
      {KeyStr("E", 4U, kTypeValue), "val"},
      {KeyStr("F", 6U, kTypeSingleDeletion), ""},
      {KeyStr("F", 5U, kTypeValue), "val"},
      {KeyStr("F", 4U, kTypeSingleDeletion), ""},
      {KeyStr("F", 3U, kTypeValue), "val"},
      {KeyStr("G", 12U, kTypeSingleDeletion), ""},
      {KeyStr("H", 6U, kTypeSingleDeletion), ""},
      {KeyStr("H", 5U, kTypeValue), "val"},
      {KeyStr("H", 4U, kTypeSingleDeletion), ""},
      {KeyStr("H", 3U, kTypeValue), "val"},
      {KeyStr("I", 12U, kTypeSingleDeletion), ""},
      {KeyStr("I", 11U, kTypeValue), "val"},
      {KeyStr("J", 6U, kTypeSingleDeletion), ""},
      {KeyStr("J", 5U, kTypeSingleDeletion), ""},
      {KeyStr("J", 4U, kTypeValue), "val"},
      {KeyStr("J", 3U, kTypeSingleDeletion), ""},
      {KeyStr("J", 2U, kTypeValue), "val"},
      {KeyStr("K", 8U, kTypeValue), "val3"},
      {KeyStr("K", 7U, kTypeValue), "val4"},
      {KeyStr("K", 6U, kTypeSingleDeletion), ""},
      {KeyStr("K", 5U, kTypeValue), "val5"},
      {KeyStr("K", 2U, kTypeSingleDeletion), ""},
      {KeyStr("K", 1U, kTypeSingleDeletion), ""},
      {KeyStr("L", 5U, kTypeSingleDeletion), ""},
      {KeyStr("L", 4U, kTypeValue), "val"},
      {KeyStr("L", 3U, kTypeDeletion), ""},
      {KeyStr("L", 2U, kTypeValue), "val"},
      {KeyStr("L", 1U, kTypeSingleDeletion), ""},
      {KeyStr("M", 10U, kTypeSingleDeletion), ""},
      {KeyStr("M", 7U, kTypeValue), "val"},
      {KeyStr("M", 5U, kTypeDeletion), ""},
      {KeyStr("M", 4U, kTypeValue), "val"},
      {KeyStr("M", 3U, kTypeSingleDeletion), ""},
A
Andres Noetzli 已提交
971 972 973 974
  });
  AddMockFile(file2);

  auto file3 = mock::MakeMockFile({
975 976 977
      {KeyStr("D", 1U, kTypeValue), "val"},
      {KeyStr("H", 1U, kTypeValue), "val"},
      {KeyStr("I", 2U, kTypeValue), "val"},
A
Andres Noetzli 已提交
978 979 980
  });
  AddMockFile(file3, 2);

981 982
  auto file4 = mock::MakeMockFile({
      {KeyStr("M", 1U, kTypeValue), "val"},
A
Andres Noetzli 已提交
983
  });
984 985 986 987 988 989 990 991 992 993 994 995 996 997 998 999 1000 1001 1002 1003 1004 1005 1006 1007 1008 1009 1010 1011
  AddMockFile(file4, 2);

  auto expected_results =
      mock::MakeMockFile({{KeyStr("A", 14U, kTypeSingleDeletion), ""},
                          {KeyStr("A", 13U, kTypeValue), ""},
                          {KeyStr("A", 12U, kTypeSingleDeletion), ""},
                          {KeyStr("A", 10U, kTypeValue), "val"},
                          {KeyStr("B", 14U, kTypeSingleDeletion), ""},
                          {KeyStr("B", 13U, kTypeValue), ""},
                          {KeyStr("C", 14U, kTypeValue), "val3"},
                          {KeyStr("D", 12U, kTypeSingleDeletion), ""},
                          {KeyStr("D", 11U, kTypeValue), ""},
                          {KeyStr("D", 10U, kTypeSingleDeletion), ""},
                          {KeyStr("E", 12U, kTypeSingleDeletion), ""},
                          {KeyStr("E", 11U, kTypeValue), ""},
                          {KeyStr("G", 15U, kTypeValue), "val"},
                          {KeyStr("G", 12U, kTypeSingleDeletion), ""},
                          {KeyStr("I", 14U, kTypeSingleDeletion), ""},
                          {KeyStr("I", 13U, kTypeValue), ""},
                          {KeyStr("J", 15U, kTypeValue), "val"},
                          {KeyStr("K", 16U, kTypeSingleDeletion), ""},
                          {KeyStr("K", 15U, kTypeValue), ""},
                          {KeyStr("K", 11U, kTypeSingleDeletion), ""},
                          {KeyStr("K", 8U, kTypeValue), "val3"},
                          {KeyStr("L", 16U, kTypeSingleDeletion), ""},
                          {KeyStr("L", 15U, kTypeValue), ""},
                          {KeyStr("M", 16U, kTypeDeletion), ""},
                          {KeyStr("M", 3U, kTypeSingleDeletion), ""}});
A
Andres Noetzli 已提交
1012 1013 1014

  SetLastSequence(22U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
1015
  RunCompaction({files}, expected_results, {10U}, 10U);
A
Andres Noetzli 已提交
1016 1017 1018 1019 1020 1021
}

// This test documents the behavior where a corrupt key follows a deletion or a
// single deletion and the (single) deletion gets removed while the corrupt key
// gets written out. TODO(noetzli): We probably want a better way to treat
// corrupt keys.
1022
TEST_F(CompactionJobTest, DISABLED_CorruptionAfterDeletion) {
A
Andres Noetzli 已提交
1023 1024 1025 1026 1027 1028 1029 1030 1031 1032 1033 1034 1035 1036 1037 1038 1039 1040
  NewDB();

  auto file1 =
      mock::MakeMockFile({{test::KeyStr("A", 6U, kTypeValue), "val3"},
                          {test::KeyStr("a", 5U, kTypeDeletion), ""},
                          {test::KeyStr("a", 4U, kTypeValue, true), "val"}});
  AddMockFile(file1);

  auto file2 =
      mock::MakeMockFile({{test::KeyStr("b", 3U, kTypeSingleDeletion), ""},
                          {test::KeyStr("b", 2U, kTypeValue, true), "val"},
                          {test::KeyStr("c", 1U, kTypeValue), "val2"}});
  AddMockFile(file2);

  auto expected_results =
      mock::MakeMockFile({{test::KeyStr("A", 0U, kTypeValue), "val3"},
                          {test::KeyStr("a", 0U, kTypeValue, true), "val"},
                          {test::KeyStr("b", 0U, kTypeValue, true), "val"},
1041
                          {test::KeyStr("c", 0U, kTypeValue), "val2"}});
A
Andres Noetzli 已提交
1042 1043 1044 1045 1046 1047

  SetLastSequence(6U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results);
}

1048 1049 1050 1051 1052 1053 1054 1055 1056 1057 1058 1059 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
TEST_F(CompactionJobTest, OldestBlobFileNumber) {
  NewDB();

  // Note: blob1 is inlined TTL, so it will not be considered for the purposes
  // of identifying the oldest referenced blob file. Similarly, blob6 will be
  // ignored because it has TTL and hence refers to a TTL blob file.
  const stl_wrappers::KVMap::value_type blob1(
      KeyStr("a", 1U, kTypeBlobIndex), BlobStrInlinedTTL("foo", 1234567890ULL));
  const stl_wrappers::KVMap::value_type blob2(KeyStr("b", 2U, kTypeBlobIndex),
                                              BlobStr(59, 123456, 999));
  const stl_wrappers::KVMap::value_type blob3(KeyStr("c", 3U, kTypeBlobIndex),
                                              BlobStr(138, 1000, 1 << 8));
  auto file1 = mock::MakeMockFile({blob1, blob2, blob3});
  AddMockFile(file1);

  const stl_wrappers::KVMap::value_type blob4(KeyStr("d", 4U, kTypeBlobIndex),
                                              BlobStr(199, 3 << 10, 1 << 20));
  const stl_wrappers::KVMap::value_type blob5(KeyStr("e", 5U, kTypeBlobIndex),
                                              BlobStr(19, 6789, 333));
  const stl_wrappers::KVMap::value_type blob6(
      KeyStr("f", 6U, kTypeBlobIndex),
      BlobStrTTL(5, 2048, 1 << 7, 1234567890ULL));
  auto file2 = mock::MakeMockFile({blob4, blob5, blob6});
  AddMockFile(file2);

  const stl_wrappers::KVMap::value_type expected_blob1(
      KeyStr("a", 0U, kTypeBlobIndex), blob1.second);
  const stl_wrappers::KVMap::value_type expected_blob2(
      KeyStr("b", 0U, kTypeBlobIndex), blob2.second);
  const stl_wrappers::KVMap::value_type expected_blob3(
      KeyStr("c", 0U, kTypeBlobIndex), blob3.second);
  const stl_wrappers::KVMap::value_type expected_blob4(
      KeyStr("d", 0U, kTypeBlobIndex), blob4.second);
  const stl_wrappers::KVMap::value_type expected_blob5(
      KeyStr("e", 0U, kTypeBlobIndex), blob5.second);
  const stl_wrappers::KVMap::value_type expected_blob6(
      KeyStr("f", 0U, kTypeBlobIndex), blob6.second);
  auto expected_results =
      mock::MakeMockFile({expected_blob1, expected_blob2, expected_blob3,
                          expected_blob4, expected_blob5, expected_blob6});

  SetLastSequence(6U);
  auto files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results, std::vector<SequenceNumber>(),
                kMaxSequenceNumber, /* output_level */ 1, /* verify */ true,
                /* expected_oldest_blob_file_number */ 19);
}

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 1123 1124 1125 1126 1127 1128 1129 1130 1131 1132 1133 1134 1135 1136 1137 1138 1139 1140 1141 1142 1143 1144 1145 1146 1147 1148 1149 1150 1151 1152 1153 1154 1155 1156 1157 1158 1159 1160 1161 1162 1163 1164 1165 1166 1167 1168 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
class CompactionJobTimestampTest : public CompactionJobTestBase {
 public:
  CompactionJobTimestampTest()
      : CompactionJobTestBase(test::PerThreadDBPath("compaction_job_ts_test"),
                              test::ComparatorWithU64Ts(), test::EncodeInt) {}
};

TEST_F(CompactionJobTimestampTest, GCDisabled) {
  NewDB();

  auto file1 =
      mock::MakeMockFile({{KeyStr("a", 10, ValueType::kTypeValue, 100), "a10"},
                          {KeyStr("a", 9, ValueType::kTypeValue, 99), "a9"},
                          {KeyStr("b", 8, ValueType::kTypeValue, 98), "b8"}});
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile(
      {{KeyStr("b", 7, ValueType::kTypeDeletionWithTimestamp, 97), ""},
       {KeyStr("c", 6, ValueType::kTypeDeletionWithTimestamp, 96), ""},
       {KeyStr("c", 5, ValueType::kTypeValue, 95), "c5"}});
  AddMockFile(file2);

  SetLastSequence(10);

  auto expected_results = mock::MakeMockFile(
      {{KeyStr("a", 10, ValueType::kTypeValue, 100), "a10"},
       {KeyStr("a", 9, ValueType::kTypeValue, 99), "a9"},
       {KeyStr("b", 8, ValueType::kTypeValue, 98), "b8"},
       {KeyStr("b", 7, ValueType::kTypeDeletionWithTimestamp, 97), ""},
       {KeyStr("c", 6, ValueType::kTypeDeletionWithTimestamp, 96), ""},
       {KeyStr("c", 5, ValueType::kTypeValue, 95), "c5"}});
  const auto& files = cfd_->current()->storage_info()->LevelFiles(0);
  RunCompaction({files}, expected_results);
}

TEST_F(CompactionJobTimestampTest, NoKeyExpired) {
  NewDB();

  auto file1 =
      mock::MakeMockFile({{KeyStr("a", 6, ValueType::kTypeValue, 100), "a6"},
                          {KeyStr("b", 7, ValueType::kTypeValue, 101), "b7"},
                          {KeyStr("c", 5, ValueType::kTypeValue, 99), "c5"}});
  AddMockFile(file1);

  auto file2 =
      mock::MakeMockFile({{KeyStr("a", 4, ValueType::kTypeValue, 98), "a4"},
                          {KeyStr("c", 3, ValueType::kTypeValue, 97), "c3"}});
  AddMockFile(file2);

  SetLastSequence(101);

  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 6, ValueType::kTypeValue, 100), "a6"},
                          {KeyStr("a", 4, ValueType::kTypeValue, 98), "a4"},
                          {KeyStr("b", 7, ValueType::kTypeValue, 101), "b7"},
                          {KeyStr("c", 5, ValueType::kTypeValue, 99), "c5"},
                          {KeyStr("c", 3, ValueType::kTypeValue, 97), "c3"}});
  const auto& files = cfd_->current()->storage_info()->LevelFiles(0);

  full_history_ts_low_ = encode_u64_ts_(0);
  RunCompaction({files}, expected_results);
}

TEST_F(CompactionJobTimestampTest, AllKeysExpired) {
  NewDB();

  auto file1 = mock::MakeMockFile(
      {{KeyStr("a", 5, ValueType::kTypeDeletionWithTimestamp, 100), ""},
       {KeyStr("b", 6, ValueType::kTypeValue, 99), "b6"}});
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile(
      {{KeyStr("a", 4, ValueType::kTypeValue, 98), "a4"},
       {KeyStr("b", 3, ValueType::kTypeDeletionWithTimestamp, 97), ""},
       {KeyStr("b", 2, ValueType::kTypeValue, 96), "b2"}});
  AddMockFile(file2);

  SetLastSequence(6);

  auto expected_results =
      mock::MakeMockFile({{KeyStr("b", 0, ValueType::kTypeValue, 0), "b6"}});
  const auto& files = cfd_->current()->storage_info()->LevelFiles(0);

  full_history_ts_low_ = encode_u64_ts_(std::numeric_limits<uint64_t>::max());
  RunCompaction({files}, expected_results);
}

TEST_F(CompactionJobTimestampTest, SomeKeysExpired) {
  NewDB();

  auto file1 =
      mock::MakeMockFile({{KeyStr("a", 5, ValueType::kTypeValue, 50), "a5"},
                          {KeyStr("b", 6, ValueType::kTypeValue, 49), "b6"}});
  AddMockFile(file1);

  auto file2 = mock::MakeMockFile(
      {{KeyStr("a", 3, ValueType::kTypeValue, 48), "a3"},
       {KeyStr("a", 2, ValueType::kTypeValue, 46), "a2"},
       {KeyStr("b", 4, ValueType::kTypeDeletionWithTimestamp, 47), ""}});
  AddMockFile(file2);

  SetLastSequence(6);

  auto expected_results =
      mock::MakeMockFile({{KeyStr("a", 5, ValueType::kTypeValue, 50), "a5"},
                          {KeyStr("b", 6, ValueType::kTypeValue, 49), "b6"}});
  const auto& files = cfd_->current()->storage_info()->LevelFiles(0);

  full_history_ts_low_ = encode_u64_ts_(49);
  RunCompaction({files}, expected_results);
}

1208
}  // namespace ROCKSDB_NAMESPACE
I
Igor Canadi 已提交
1209

I
Igor Sugak 已提交
1210 1211 1212 1213
int main(int argc, char** argv) {
  ::testing::InitGoogleTest(&argc, argv);
  return RUN_ALL_TESTS();
}
1214 1215 1216 1217

#else
#include <stdio.h>

1218
int main(int /*argc*/, char** /*argv*/) {
1219 1220 1221 1222 1223 1224
  fprintf(stderr,
          "SKIPPED as CompactionJobStats is not supported in ROCKSDB_LITE\n");
  return 0;
}

#endif  // ROCKSDB_LITE