You can not select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
rocksdb/db/range_del_aggregator.h

442 lines
13 KiB

// Copyright (c) 2018-present, Facebook, Inc. All rights reserved.
// 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).
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
#pragma once
#include <algorithm>
#include <iterator>
#include <list>
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
#include <map>
Recommit "Avoid adding tombstones of the same file to RangeDelAggregator multiple times" Summary: The origin commit #3635 will hurt performance for users who aren't using range deletions, because unneeded std::set operations, so it was reverted by commit 44653c7b7aabe821e671946e732dda7ae6b43d1b. (see #3672) To fix this, move the set to and add a check in , i.e., file will be added only if is non-nullptr. The db_bench command which find the performance regression: > ./db_bench --benchmarks=fillrandom,seekrandomwhilewriting --threads=1 --num=1000000 --reads=150000 --key_size=66 > --value_size=1262 --statistics=0 --compression_ratio=0.5 --histogram=1 --seek_nexts=1 --stats_per_interval=1 > --stats_interval_seconds=600 --max_background_flushes=4 --num_multi_db=1 --max_background_compactions=16 --seed=1522388277 > -write_buffer_size=1048576 --level0_file_num_compaction_trigger=10000 --compression_type=none Before and after the modification, I re-run this command on the machine, the results of are as follows: **fillrandom** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 5.92 | 8.57 | 19.63 | 980.97 | 12196.00 | after commit | 5.91 | 8.55 | 19.34 | 965.56 | 13513.56 | **seekrandomwhilewriting** Table | P50 | P75 | P99 | P99.9 | P99.99 | ---- | --- | --- | --- | ----- | ------ | before commit | 1418.62 | 1867.01 | 3823.28 | 4980.99 | 9240.00 | after commit | 1450.54 | 1880.61 | 3962.87 | 5429.60 | 7542.86 | Closes https://github.com/facebook/rocksdb/pull/3800 Differential Revision: D7874245 Pulled By: ajkr fbshipit-source-id: 2e8bec781b3f7399246babd66395c88619534a17
6 years ago
#include <set>
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
#include <string>
#include <vector>
#include "db/compaction/compaction_iteration_stats.h"
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
#include "db/dbformat.h"
#include "db/pinned_iterators_manager.h"
#include "db/range_del_aggregator.h"
#include "db/range_tombstone_fragmenter.h"
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
#include "db/version_edit.h"
#include "rocksdb/comparator.h"
#include "rocksdb/types.h"
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
#include "table/internal_iterator.h"
#include "table/scoped_arena_iterator.h"
#include "table/table_builder.h"
#include "util/heap.h"
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
#include "util/kv_map.h"
namespace ROCKSDB_NAMESPACE {
class TruncatedRangeDelIterator {
public:
TruncatedRangeDelIterator(
std::unique_ptr<FragmentedRangeTombstoneIterator> iter,
const InternalKeyComparator* icmp, const InternalKey* smallest,
const InternalKey* largest);
bool Valid() const;
void Next();
void Prev();
void InternalNext();
// Seeks to the tombstone with the highest viisble sequence number that covers
// target (a user key). If no such tombstone exists, the position will be at
// the earliest tombstone that ends after target.
void Seek(const Slice& target);
// Seeks to the tombstone with the highest viisble sequence number that covers
// target (a user key). If no such tombstone exists, the position will be at
// the latest tombstone that starts before target.
void SeekForPrev(const Slice& target);
void SeekToFirst();
void SeekToLast();
ParsedInternalKey start_key() const {
return (smallest_ == nullptr ||
icmp_->Compare(*smallest_, iter_->parsed_start_key()) <= 0)
? iter_->parsed_start_key()
: *smallest_;
}
ParsedInternalKey end_key() const {
return (largest_ == nullptr ||
icmp_->Compare(iter_->parsed_end_key(), *largest_) <= 0)
? iter_->parsed_end_key()
: *largest_;
}
SequenceNumber seq() const { return iter_->seq(); }
std::map<SequenceNumber, std::unique_ptr<TruncatedRangeDelIterator>>
SplitBySnapshot(const std::vector<SequenceNumber>& snapshots);
SequenceNumber upper_bound() const { return iter_->upper_bound(); }
SequenceNumber lower_bound() const { return iter_->lower_bound(); }
private:
std::unique_ptr<FragmentedRangeTombstoneIterator> iter_;
const InternalKeyComparator* icmp_;
const ParsedInternalKey* smallest_ = nullptr;
const ParsedInternalKey* largest_ = nullptr;
std::list<ParsedInternalKey> pinned_bounds_;
const InternalKey* smallest_ikey_;
const InternalKey* largest_ikey_;
};
struct SeqMaxComparator {
bool operator()(const TruncatedRangeDelIterator* a,
const TruncatedRangeDelIterator* b) const {
return a->seq() > b->seq();
}
};
struct StartKeyMinComparator {
explicit StartKeyMinComparator(const InternalKeyComparator* c) : icmp(c) {}
bool operator()(const TruncatedRangeDelIterator* a,
const TruncatedRangeDelIterator* b) const {
return icmp->Compare(a->start_key(), b->start_key()) > 0;
}
const InternalKeyComparator* icmp;
};
class ForwardRangeDelIterator {
public:
explicit ForwardRangeDelIterator(const InternalKeyComparator* icmp);
bool ShouldDelete(const ParsedInternalKey& parsed);
void Invalidate();
void AddNewIter(TruncatedRangeDelIterator* iter,
const ParsedInternalKey& parsed) {
iter->Seek(parsed.user_key);
PushIter(iter, parsed);
assert(active_iters_.size() == active_seqnums_.size());
}
size_t UnusedIdx() const { return unused_idx_; }
void IncUnusedIdx() { unused_idx_++; }
private:
using ActiveSeqSet =
std::multiset<TruncatedRangeDelIterator*, SeqMaxComparator>;
struct EndKeyMinComparator {
explicit EndKeyMinComparator(const InternalKeyComparator* c) : icmp(c) {}
bool operator()(const ActiveSeqSet::const_iterator& a,
const ActiveSeqSet::const_iterator& b) const {
return icmp->Compare((*a)->end_key(), (*b)->end_key()) > 0;
}
const InternalKeyComparator* icmp;
};
void PushIter(TruncatedRangeDelIterator* iter,
const ParsedInternalKey& parsed) {
if (!iter->Valid()) {
// The iterator has been fully consumed, so we don't need to add it to
// either of the heaps.
return;
}
int cmp = icmp_->Compare(parsed, iter->start_key());
if (cmp < 0) {
PushInactiveIter(iter);
} else {
PushActiveIter(iter);
}
}
void PushActiveIter(TruncatedRangeDelIterator* iter) {
auto seq_pos = active_seqnums_.insert(iter);
active_iters_.push(seq_pos);
}
TruncatedRangeDelIterator* PopActiveIter() {
auto active_top = active_iters_.top();
auto iter = *active_top;
active_iters_.pop();
active_seqnums_.erase(active_top);
return iter;
}
void PushInactiveIter(TruncatedRangeDelIterator* iter) {
inactive_iters_.push(iter);
}
TruncatedRangeDelIterator* PopInactiveIter() {
auto* iter = inactive_iters_.top();
inactive_iters_.pop();
return iter;
}
const InternalKeyComparator* icmp_;
size_t unused_idx_;
ActiveSeqSet active_seqnums_;
BinaryHeap<ActiveSeqSet::const_iterator, EndKeyMinComparator> active_iters_;
BinaryHeap<TruncatedRangeDelIterator*, StartKeyMinComparator> inactive_iters_;
};
class ReverseRangeDelIterator {
public:
explicit ReverseRangeDelIterator(const InternalKeyComparator* icmp);
bool ShouldDelete(const ParsedInternalKey& parsed);
void Invalidate();
void AddNewIter(TruncatedRangeDelIterator* iter,
const ParsedInternalKey& parsed) {
iter->SeekForPrev(parsed.user_key);
PushIter(iter, parsed);
assert(active_iters_.size() == active_seqnums_.size());
}
size_t UnusedIdx() const { return unused_idx_; }
void IncUnusedIdx() { unused_idx_++; }
private:
using ActiveSeqSet =
std::multiset<TruncatedRangeDelIterator*, SeqMaxComparator>;
struct EndKeyMaxComparator {
explicit EndKeyMaxComparator(const InternalKeyComparator* c) : icmp(c) {}
bool operator()(const TruncatedRangeDelIterator* a,
const TruncatedRangeDelIterator* b) const {
return icmp->Compare(a->end_key(), b->end_key()) < 0;
}
const InternalKeyComparator* icmp;
};
struct StartKeyMaxComparator {
explicit StartKeyMaxComparator(const InternalKeyComparator* c) : icmp(c) {}
bool operator()(const ActiveSeqSet::const_iterator& a,
const ActiveSeqSet::const_iterator& b) const {
return icmp->Compare((*a)->start_key(), (*b)->start_key()) < 0;
}
const InternalKeyComparator* icmp;
};
void PushIter(TruncatedRangeDelIterator* iter,
const ParsedInternalKey& parsed) {
if (!iter->Valid()) {
// The iterator has been fully consumed, so we don't need to add it to
// either of the heaps.
} else if (icmp_->Compare(iter->end_key(), parsed) <= 0) {
PushInactiveIter(iter);
} else {
PushActiveIter(iter);
}
}
void PushActiveIter(TruncatedRangeDelIterator* iter) {
auto seq_pos = active_seqnums_.insert(iter);
active_iters_.push(seq_pos);
}
TruncatedRangeDelIterator* PopActiveIter() {
auto active_top = active_iters_.top();
auto iter = *active_top;
active_iters_.pop();
active_seqnums_.erase(active_top);
return iter;
}
void PushInactiveIter(TruncatedRangeDelIterator* iter) {
inactive_iters_.push(iter);
}
TruncatedRangeDelIterator* PopInactiveIter() {
auto* iter = inactive_iters_.top();
inactive_iters_.pop();
return iter;
}
const InternalKeyComparator* icmp_;
size_t unused_idx_;
ActiveSeqSet active_seqnums_;
BinaryHeap<ActiveSeqSet::const_iterator, StartKeyMaxComparator> active_iters_;
BinaryHeap<TruncatedRangeDelIterator*, EndKeyMaxComparator> inactive_iters_;
};
enum class RangeDelPositioningMode { kForwardTraversal, kBackwardTraversal };
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
class RangeDelAggregator {
public:
explicit RangeDelAggregator(const InternalKeyComparator* icmp)
: icmp_(icmp) {}
virtual ~RangeDelAggregator() {}
virtual void AddTombstones(
std::unique_ptr<FragmentedRangeTombstoneIterator> input_iter,
const InternalKey* smallest = nullptr,
const InternalKey* largest = nullptr) = 0;
bool ShouldDelete(const Slice& key, RangeDelPositioningMode mode) {
ParsedInternalKey parsed;
if (!ParseInternalKey(key, &parsed)) {
return false;
}
return ShouldDelete(parsed, mode);
}
virtual bool ShouldDelete(const ParsedInternalKey& parsed,
RangeDelPositioningMode mode) = 0;
virtual void InvalidateRangeDelMapPositions() = 0;
virtual bool IsEmpty() const = 0;
bool AddFile(uint64_t file_number) {
return files_seen_.insert(file_number).second;
}
protected:
class StripeRep {
public:
StripeRep(const InternalKeyComparator* icmp, SequenceNumber upper_bound,
SequenceNumber lower_bound)
: icmp_(icmp),
forward_iter_(icmp),
reverse_iter_(icmp),
upper_bound_(upper_bound),
lower_bound_(lower_bound) {}
void AddTombstones(std::unique_ptr<TruncatedRangeDelIterator> input_iter) {
iters_.push_back(std::move(input_iter));
}
bool IsEmpty() const { return iters_.empty(); }
bool ShouldDelete(const ParsedInternalKey& parsed,
RangeDelPositioningMode mode);
void Invalidate() {
if (!IsEmpty()) {
InvalidateForwardIter();
InvalidateReverseIter();
}
}
bool IsRangeOverlapped(const Slice& start, const Slice& end);
private:
bool InStripe(SequenceNumber seq) const {
return lower_bound_ <= seq && seq <= upper_bound_;
}
void InvalidateForwardIter() { forward_iter_.Invalidate(); }
void InvalidateReverseIter() { reverse_iter_.Invalidate(); }
const InternalKeyComparator* icmp_;
std::vector<std::unique_ptr<TruncatedRangeDelIterator>> iters_;
ForwardRangeDelIterator forward_iter_;
ReverseRangeDelIterator reverse_iter_;
SequenceNumber upper_bound_;
SequenceNumber lower_bound_;
};
const InternalKeyComparator* icmp_;
private:
std::set<uint64_t> files_seen_;
};
class ReadRangeDelAggregator final : public RangeDelAggregator {
public:
ReadRangeDelAggregator(const InternalKeyComparator* icmp,
SequenceNumber upper_bound)
: RangeDelAggregator(icmp),
rep_(icmp, upper_bound, 0 /* lower_bound */) {}
~ReadRangeDelAggregator() override {}
using RangeDelAggregator::ShouldDelete;
void AddTombstones(
std::unique_ptr<FragmentedRangeTombstoneIterator> input_iter,
const InternalKey* smallest = nullptr,
const InternalKey* largest = nullptr) override;
bool ShouldDelete(const ParsedInternalKey& parsed,
RangeDelPositioningMode mode) final override {
if (rep_.IsEmpty()) {
return false;
}
return ShouldDeleteImpl(parsed, mode);
}
bool IsRangeOverlapped(const Slice& start, const Slice& end);
void InvalidateRangeDelMapPositions() override { rep_.Invalidate(); }
bool IsEmpty() const override { return rep_.IsEmpty(); }
private:
StripeRep rep_;
bool ShouldDeleteImpl(const ParsedInternalKey& parsed,
RangeDelPositioningMode mode);
};
class CompactionRangeDelAggregator : public RangeDelAggregator {
public:
CompactionRangeDelAggregator(const InternalKeyComparator* icmp,
const std::vector<SequenceNumber>& snapshots)
: RangeDelAggregator(icmp), snapshots_(&snapshots) {}
~CompactionRangeDelAggregator() override {}
void AddTombstones(
std::unique_ptr<FragmentedRangeTombstoneIterator> input_iter,
const InternalKey* smallest = nullptr,
const InternalKey* largest = nullptr) override;
using RangeDelAggregator::ShouldDelete;
bool ShouldDelete(const ParsedInternalKey& parsed,
RangeDelPositioningMode mode) override;
bool IsRangeOverlapped(const Slice& start, const Slice& end);
void InvalidateRangeDelMapPositions() override {
for (auto& rep : reps_) {
rep.second.Invalidate();
}
}
bool IsEmpty() const override {
for (const auto& rep : reps_) {
if (!rep.second.IsEmpty()) {
return false;
}
}
return true;
}
// Creates an iterator over all the range tombstones in the aggregator, for
// use in compaction. Nullptr arguments indicate that the iterator range is
// unbounded.
// NOTE: the boundaries are used for optimization purposes to reduce the
// number of tombstones that are passed to the fragmenter; they do not
// guarantee that the resulting iterator only contains range tombstones that
// cover keys in the provided range. If required, these bounds must be
// enforced during iteration.
std::unique_ptr<FragmentedRangeTombstoneIterator> NewIterator(
const Slice* lower_bound = nullptr, const Slice* upper_bound = nullptr,
bool upper_bound_inclusive = false);
private:
std::vector<std::unique_ptr<TruncatedRangeDelIterator>> parent_iters_;
std::map<SequenceNumber, StripeRep> reps_;
const std::vector<SequenceNumber>* snapshots_;
Compaction Support for Range Deletion Summary: This diff introduces RangeDelAggregator, which takes ownership of iterators provided to it via AddTombstones(). The tombstones are organized in a two-level map (snapshot stripe -> begin key -> tombstone). Tombstone creation avoids data copy by holding Slices returned by the iterator, which remain valid thanks to pinning. For compaction, we create a hierarchical range tombstone iterator with structure matching the iterator over compaction input data. An aggregator based on that iterator is used by CompactionIterator to determine which keys are covered by range tombstones. In case of merge operand, the same aggregator is used by MergeHelper. Upon finishing each file in the compaction, relevant range tombstones are added to the output file's range tombstone metablock and file boundaries are updated accordingly. To check whether a key is covered by range tombstone, RangeDelAggregator::ShouldDelete() considers tombstones in the key's snapshot stripe. When this function is used outside of compaction, it also checks newer stripes, which can contain covering tombstones. Currently the intra-stripe check involves a linear scan; however, in the future we plan to collapse ranges within a stripe such that binary search can be used. RangeDelAggregator::AddToBuilder() adds all range tombstones in the table's key-range to a new table's range tombstone meta-block. Since range tombstones may fall in the gap between files, we may need to extend some files' key-ranges. The strategy is (1) first file extends as far left as possible and other files do not extend left, (2) all files extend right until either the start of the next file or the end of the last range tombstone in the gap, whichever comes first. One other notable change is adding release/move semantics to ScopedArenaIterator such that it can be used to transfer ownership of an arena-allocated iterator, similar to how unique_ptr is used for malloc'd data. Depends on D61473 Test Plan: compaction_iterator_test, mock_table, end-to-end tests in D63927 Reviewers: sdong, IslamAbdelRahman, wanning, yhchiang, lightmark Reviewed By: lightmark Subscribers: andrewkr, dhruba, leveldb Differential Revision: https://reviews.facebook.net/D62205
8 years ago
};
} // namespace ROCKSDB_NAMESPACE