compaction assertion triggering test fix for sequence zeroing assertion trip

main
Reid Horuff 9 years ago
parent 521da3abb3
commit 97ea8afaaf
  1. 3
      db/compaction.cc
  2. 5
      db/compaction.h
  3. 4
      db/compaction_iterator.cc
  4. 13
      db/compaction_job_test.cc
  5. 2
      db/db_compaction_filter_test.cc
  6. 80
      db/db_compaction_test.cc
  7. 2
      table/cuckoo_table_builder.cc

@ -184,6 +184,9 @@ Compaction::Compaction(VersionStorageInfo* vstorage,
&arena_);
}
}
Slice smallest_user_key;
GetBoundaryKeys(vstorage, inputs_, &smallest_user_key, &largest_user_key_);
}
Compaction::~Compaction() {

@ -218,6 +218,8 @@ class Compaction {
output_table_properties_ = std::move(tp);
}
Slice GetLargestUserKey() const { return largest_user_key_; }
private:
// mark (or clear) all files that are being compacted
void MarkFilesBeingCompacted(bool mark_as_compacted);
@ -284,6 +286,9 @@ class Compaction {
// table properties of output files
TablePropertiesCollection output_table_properties_;
// largest user keys in compaction
Slice largest_user_key_;
};
// Utility function

@ -406,12 +406,14 @@ void CompactionIterator::PrepareOutput() {
// Zeroing out the sequence number leads to better compression.
// If this is the bottommost level (no files in lower levels)
// and the earliest snapshot is larger than this seqno
// and the userkey differs from the last userkey in compaction
// then we can squash the seqno to zero.
// This is safe for TransactionDB write-conflict checking since transactions
// only care about sequence number larger than any active snapshots.
if (bottommost_level_ && valid_ && ikey_.sequence < earliest_snapshot_ &&
ikey_.type != kTypeMerge) {
ikey_.type != kTypeMerge &&
!cmp_->Equal(compaction_->GetLargestUserKey(), ikey_.user_key)) {
assert(ikey_.type != kTypeDeletion && ikey_.type != kTypeSingleDeletion);
ikey_.sequence = 0;
current_key_.UpdateInternalKey(0, ikey_.type);

@ -162,9 +162,12 @@ class CompactionJobTest : public testing::Test {
auto key = ToString(i * kMatchingKeys + k);
auto value = ToString(i * kKeysPerFile + k);
InternalKey internal_key(key, ++sequence_number, kTypeValue);
// This is how the key will look like once it's written in bottommost
// file
InternalKey bottommost_internal_key(key, 0, kTypeValue);
InternalKey bottommost_internal_key(
key, (key == "9999") ? sequence_number : 0, kTypeValue);
if (corrupt_id(k)) {
test::CorruptKeyType(&internal_key);
test::CorruptKeyType(&bottommost_internal_key);
@ -348,7 +351,7 @@ TEST_F(CompactionJobTest, SimpleOverwrite) {
auto expected_results =
mock::MakeMockFile({{KeyStr("a", 0U, kTypeValue), "val2"},
{KeyStr("b", 0U, kTypeValue), "val3"}});
{KeyStr("b", 4U, kTypeValue), "val3"}});
SetLastSequence(4U);
auto files = cfd_->current()->storage_info()->LevelFiles(0);
@ -401,7 +404,7 @@ TEST_F(CompactionJobTest, SimpleMerge) {
auto expected_results =
mock::MakeMockFile({{KeyStr("a", 0U, kTypeValue), "3,4,5"},
{KeyStr("b", 0U, kTypeValue), "1,2"}});
{KeyStr("b", 2U, kTypeValue), "1,2"}});
SetLastSequence(5U);
auto files = cfd_->current()->storage_info()->LevelFiles(0);
@ -716,7 +719,7 @@ TEST_F(CompactionJobTest, SingleDeleteZeroSeq) {
AddMockFile(file2);
auto expected_results = mock::MakeMockFile({
{KeyStr("dummy", 0U, kTypeValue), "val2"},
{KeyStr("dummy", 5U, kTypeValue), "val2"},
});
SetLastSequence(22U);
@ -900,7 +903,7 @@ TEST_F(CompactionJobTest, CorruptionAfterDeletion) {
mock::MakeMockFile({{test::KeyStr("A", 0U, kTypeValue), "val3"},
{test::KeyStr("a", 0U, kTypeValue, true), "val"},
{test::KeyStr("b", 0U, kTypeValue, true), "val"},
{test::KeyStr("c", 0U, kTypeValue), "val2"}});
{test::KeyStr("c", 1U, kTypeValue), "val2"}});
SetLastSequence(6U);
auto files = cfd_->current()->storage_info()->LevelFiles(0);

@ -583,7 +583,7 @@ TEST_F(DBTestCompactionFilter, CompactionFilterContextManual) {
iter->Next();
}
ASSERT_EQ(total, 700);
ASSERT_EQ(count, 1);
ASSERT_EQ(count, 2);
}
}
#endif // ROCKSDB_LITE

@ -68,6 +68,32 @@ class OnFileDeletionListener : public EventListener {
std::string expected_file_name_;
};
class FlushedFileCollector : public EventListener {
public:
FlushedFileCollector() {}
~FlushedFileCollector() {}
virtual void OnFlushCompleted(DB* db, const FlushJobInfo& info) override {
std::lock_guard<std::mutex> lock(mutex_);
flushed_files_.push_back(info.file_path);
}
std::vector<std::string> GetFlushedFiles() {
std::lock_guard<std::mutex> lock(mutex_);
std::vector<std::string> result;
for (auto fname : flushed_files_) {
result.push_back(fname);
}
return result;
}
void ClearFlushedFiles() { flushed_files_.clear(); }
private:
std::vector<std::string> flushed_files_;
std::mutex mutex_;
};
static const int kCDTValueSize = 1000;
static const int kCDTKeysPerBuffer = 4;
static const int kCDTNumLevels = 8;
@ -563,6 +589,60 @@ TEST_F(DBCompactionTest, MinorCompactionsHappen) {
} while (ChangeCompactOptions());
}
TEST_F(DBCompactionTest, ZeroSeqIdCompaction) {
Options options;
options.compaction_style = kCompactionStyleLevel;
options.level0_file_num_compaction_trigger = 3;
FlushedFileCollector* collector = new FlushedFileCollector();
options.listeners.emplace_back(collector);
// compaction options
CompactionOptions compact_opt;
compact_opt.compression = kNoCompression;
compact_opt.output_file_size_limit = 4096;
const int key_len = compact_opt.output_file_size_limit / 5;
options = CurrentOptions(options);
DestroyAndReopen(options);
std::vector<const Snapshot*> snaps;
// create first file and flush to l0
for (auto& key : {"1", "2", "3", "3", "3", "3"}) {
Put(key, std::string(key_len, 'A'));
snaps.push_back(dbfull()->GetSnapshot());
}
Flush();
dbfull()->TEST_WaitForFlushMemTable();
// create second file and flush to l0
for (auto& key : {"3", "4", "5", "6", "7", "8"}) {
Put(key, std::string(key_len, 'A'));
snaps.push_back(dbfull()->GetSnapshot());
}
Flush();
dbfull()->TEST_WaitForFlushMemTable();
// move both files down to l1
dbfull()->CompactFiles(compact_opt, collector->GetFlushedFiles(), 1);
// release snap so that first instance of key(3) can have seqId=0
for (auto snap : snaps) {
dbfull()->ReleaseSnapshot(snap);
}
// create 3 files in l0 so to trigger compaction
for (int i = 0; i < options.level0_file_num_compaction_trigger; i++) {
Put("2", std::string(1, 'A'));
Flush();
dbfull()->TEST_WaitForFlushMemTable();
}
dbfull()->TEST_WaitForCompact();
ASSERT_OK(Put("", ""));
}
// Check that writes done during a memtable compaction are recovered
// if the database is shutdown during the memtable compaction.
TEST_F(DBCompactionTest, RecoverDuringMemtableCompaction) {

@ -109,8 +109,6 @@ void CuckooTableBuilder::Add(const Slice& key, const Slice& value) {
status_ = Status::NotSupported("all keys have to be the same size");
return;
}
// Even if one sequence number is non-zero, then it is not last level.
assert(!is_last_level_file_ || ikey.sequence == 0);
if (ikey.type == kTypeValue) {
if (!has_seen_first_value_) {

Loading…
Cancel
Save