2018-12-18 02:26:56 +01:00
|
|
|
// Copyright (c) 2018-present, Facebook, Inc. All rights reserved.
|
2017-07-16 01:03:42 +02:00
|
|
|
// 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
2016-10-18 21:04:56 +02:00
|
|
|
|
|
|
|
#pragma once
|
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
#include <algorithm>
|
|
|
|
#include <iterator>
|
2018-07-14 07:40:23 +02:00
|
|
|
#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
2016-10-18 21:04:56 +02:00
|
|
|
#include <map>
|
2018-05-05 01:37:39 +02:00
|
|
|
#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
2016-10-18 21:04:56 +02:00
|
|
|
#include <string>
|
|
|
|
#include <vector>
|
|
|
|
|
2019-05-31 20:52:59 +02:00
|
|
|
#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
2016-10-18 21:04:56 +02:00
|
|
|
#include "db/dbformat.h"
|
|
|
|
#include "db/pinned_iterators_manager.h"
|
2018-12-18 02:26:56 +01:00
|
|
|
#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
2016-10-18 21:04:56 +02:00
|
|
|
#include "db/version_edit.h"
|
2020-01-22 01:11:08 +01:00
|
|
|
#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
2016-10-18 21:04:56 +02:00
|
|
|
#include "table/internal_iterator.h"
|
|
|
|
#include "table/scoped_arena_iterator.h"
|
|
|
|
#include "table/table_builder.h"
|
2018-12-18 02:26:56 +01:00
|
|
|
#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
2016-10-18 21:04:56 +02:00
|
|
|
#include "util/kv_map.h"
|
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
namespace ROCKSDB_NAMESPACE {
|
2016-10-29 00:44:48 +02:00
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
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();
|
|
|
|
}
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
};
|
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
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;
|
2018-10-10 00:15:27 +02:00
|
|
|
};
|
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
class ForwardRangeDelIterator {
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
public:
|
2018-12-18 23:10:31 +01:00
|
|
|
explicit ForwardRangeDelIterator(const InternalKeyComparator* icmp);
|
2018-12-18 02:26:56 +01:00
|
|
|
|
|
|
|
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_;
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
};
|
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
class ReverseRangeDelIterator {
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
public:
|
2018-12-18 23:10:31 +01:00
|
|
|
explicit ReverseRangeDelIterator(const InternalKeyComparator* icmp);
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
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;
|
|
|
|
}
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
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;
|
|
|
|
}
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
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_;
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
};
|
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
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
2016-10-18 21:04:56 +02:00
|
|
|
class RangeDelAggregator {
|
|
|
|
public:
|
2018-12-18 02:26:56 +01:00
|
|
|
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;
|
2020-10-01 04:15:42 +02:00
|
|
|
|
|
|
|
Status pikStatus = ParseInternalKey(key, &parsed);
|
|
|
|
assert(pikStatus.ok());
|
|
|
|
if (!pikStatus.ok()) {
|
2017-09-15 00:41:19 +02:00
|
|
|
return false;
|
|
|
|
}
|
2020-10-01 04:15:42 +02:00
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
return ShouldDelete(parsed, mode);
|
2017-09-15 00:41:19 +02:00
|
|
|
}
|
2018-12-18 02:26:56 +01:00
|
|
|
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),
|
2018-12-18 23:10:31 +01:00
|
|
|
forward_iter_(icmp),
|
|
|
|
reverse_iter_(icmp),
|
2018-12-18 02:26:56 +01:00
|
|
|
upper_bound_(upper_bound),
|
|
|
|
lower_bound_(lower_bound) {}
|
|
|
|
|
|
|
|
void AddTombstones(std::unique_ptr<TruncatedRangeDelIterator> input_iter) {
|
|
|
|
iters_.push_back(std::move(input_iter));
|
2017-09-15 00:41:19 +02:00
|
|
|
}
|
2018-12-18 02:26:56 +01:00
|
|
|
|
|
|
|
bool IsEmpty() const { return iters_.empty(); }
|
|
|
|
|
|
|
|
bool ShouldDelete(const ParsedInternalKey& parsed,
|
|
|
|
RangeDelPositioningMode mode);
|
|
|
|
|
|
|
|
void Invalidate() {
|
2019-05-17 00:20:19 +02:00
|
|
|
if (!IsEmpty()) {
|
|
|
|
InvalidateForwardIter();
|
|
|
|
InvalidateReverseIter();
|
|
|
|
}
|
2018-12-18 02:26:56 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
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_;
|
|
|
|
};
|
|
|
|
|
2019-04-18 21:22:29 +02:00
|
|
|
class ReadRangeDelAggregator final : public RangeDelAggregator {
|
2018-12-18 02:26:56 +01:00
|
|
|
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,
|
2019-04-18 21:22:29 +02:00
|
|
|
RangeDelPositioningMode mode) final override {
|
|
|
|
if (rep_.IsEmpty()) {
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
return ShouldDeleteImpl(parsed, mode);
|
|
|
|
}
|
2018-12-18 02:26:56 +01:00
|
|
|
|
2017-11-28 20:18:42 +01:00
|
|
|
bool IsRangeOverlapped(const Slice& start, const Slice& end);
|
|
|
|
|
2018-12-18 02:26:56 +01:00
|
|
|
void InvalidateRangeDelMapPositions() override { rep_.Invalidate(); }
|
|
|
|
|
|
|
|
bool IsEmpty() const override { return rep_.IsEmpty(); }
|
Maintain position in range deletions map
Summary:
When deletion-collapsing mode is enabled (i.e., for DBIter/CompactionIterator), we maintain position in the tombstone maps across calls to ShouldDelete(). Since iterators often access keys sequentially (or reverse-sequentially), scanning forward/backward from the last position can be faster than binary-searching the map for every key.
- When Next() is invoked on an iterator, we use kForwardTraversal to scan forwards, if needed, until arriving at the range deletion containing the next key.
- Similarly for Prev(), we use kBackwardTraversal to scan backwards in the range deletion map.
- When the iterator seeks, we use kBinarySearch for repositioning
- After tombstones are added or before the first ShouldDelete() invocation, the current position is set to invalid, which forces kBinarySearch to be used.
- Non-iterator users (i.e., Get()) use kFullScan, which has the same behavior as before---scan the whole map for every key passed to ShouldDelete().
Closes https://github.com/facebook/rocksdb/pull/1701
Differential Revision: D4350318
Pulled By: ajkr
fbshipit-source-id: 5129b76
2017-01-05 19:22:46 +01:00
|
|
|
|
Range deletion performance improvements + cleanup (#4014)
Summary:
This fixes the same performance issue that #3992 fixes but with much more invasive cleanup.
I'm more excited about this PR because it paves the way for fixing another problem we uncovered at Cockroach where range deletion tombstones can cause massive compactions. For example, suppose L4 contains deletions from [a, c) and [x, z) and no other keys, and L5 is entirely empty. L6, however, is full of data. When compacting L4 -> L5, we'll end up with one file that spans, massively, from [a, z). When we go to compact L5 -> L6, we'll have to rewrite all of L6! If, instead of range deletions in L4, we had keys a, b, x, y, and z, RocksDB would have been smart enough to create two files in L5: one for a and b and another for x, y, and z.
With the changes in this PR, it will be possible to adjust the compaction logic to split tombstones/start new output files when they would span too many files in the grandparent level.
ajkr please take a look when you have a minute!
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4014
Differential Revision: D8773253
Pulled By: ajkr
fbshipit-source-id: ec62fa85f648fdebe1380b83ed997f9baec35677
2018-07-12 23:28:10 +02:00
|
|
|
private:
|
2018-12-18 02:26:56 +01:00
|
|
|
StripeRep rep_;
|
2019-04-18 21:22:29 +02:00
|
|
|
|
|
|
|
bool ShouldDeleteImpl(const ParsedInternalKey& parsed,
|
|
|
|
RangeDelPositioningMode mode);
|
2018-12-18 02:26:56 +01:00
|
|
|
};
|
|
|
|
|
|
|
|
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
2016-10-18 21:04:56 +02:00
|
|
|
};
|
2016-11-19 23:14:35 +01:00
|
|
|
|
2020-02-20 21:07:53 +01:00
|
|
|
} // namespace ROCKSDB_NAMESPACE
|