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
|
|
|
// Copyright (c) 2016-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
|
|
|
|
|
|
|
#include "db/range_del_aggregator.h"
|
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
|
|
|
#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 <algorithm>
|
|
|
|
|
|
|
|
namespace rocksdb {
|
|
|
|
|
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
|
|
|
struct TombstoneStartKeyComparator {
|
2018-10-10 00:15:27 +02:00
|
|
|
explicit TombstoneStartKeyComparator(const InternalKeyComparator* c)
|
|
|
|
: cmp(c) {}
|
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-10-10 00:15:27 +02:00
|
|
|
bool operator()(const TruncatedRangeTombstone& a,
|
|
|
|
const TruncatedRangeTombstone& b) const {
|
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
|
|
|
return cmp->Compare(a.start_key_, b.start_key_) < 0;
|
|
|
|
}
|
|
|
|
|
2018-10-10 00:15:27 +02:00
|
|
|
const InternalKeyComparator* cmp;
|
|
|
|
};
|
|
|
|
|
|
|
|
struct ParsedInternalKeyComparator {
|
|
|
|
explicit ParsedInternalKeyComparator(const InternalKeyComparator* c)
|
|
|
|
: cmp(c) {}
|
|
|
|
|
|
|
|
bool operator()(const ParsedInternalKey& a,
|
|
|
|
const ParsedInternalKey& b) const {
|
|
|
|
return cmp->Compare(a, b) < 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
const InternalKeyComparator* cmp;
|
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
|
|
|
};
|
|
|
|
|
|
|
|
// An UncollapsedRangeDelMap is quick to create but slow to answer ShouldDelete
|
|
|
|
// queries.
|
|
|
|
class UncollapsedRangeDelMap : public RangeDelMap {
|
2018-10-10 00:15:27 +02:00
|
|
|
typedef std::multiset<TruncatedRangeTombstone, TombstoneStartKeyComparator>
|
|
|
|
Rep;
|
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
|
|
|
|
|
|
|
class Iterator : public RangeDelIterator {
|
|
|
|
const Rep& rep_;
|
|
|
|
Rep::const_iterator iter_;
|
|
|
|
|
|
|
|
public:
|
|
|
|
Iterator(const Rep& rep) : rep_(rep), iter_(rep.begin()) {}
|
|
|
|
bool Valid() const override { return iter_ != rep_.end(); }
|
|
|
|
void Next() override { iter_++; }
|
|
|
|
|
|
|
|
void Seek(const Slice&) override {
|
2018-10-10 00:15:27 +02:00
|
|
|
fprintf(stderr,
|
|
|
|
"UncollapsedRangeDelMap::Iterator::Seek(Slice&) unimplemented\n");
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
|
|
|
void Seek(const ParsedInternalKey&) override {
|
|
|
|
fprintf(stderr,
|
|
|
|
"UncollapsedRangeDelMap::Iterator::Seek(ParsedInternalKey&) "
|
|
|
|
"unimplemented\n");
|
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
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
2018-10-10 00:15:27 +02:00
|
|
|
RangeTombstone Tombstone() const override { return iter_->Tombstone(); }
|
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
|
|
|
};
|
|
|
|
|
|
|
|
Rep rep_;
|
2018-10-10 00:15:27 +02:00
|
|
|
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
|
|
|
|
|
|
|
public:
|
2018-10-10 00:15:27 +02:00
|
|
|
explicit UncollapsedRangeDelMap(const InternalKeyComparator* icmp)
|
|
|
|
: rep_(TombstoneStartKeyComparator(icmp)), icmp_(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
|
|
|
|
|
|
|
bool ShouldDelete(const ParsedInternalKey& parsed,
|
2018-08-15 00:03:57 +02:00
|
|
|
RangeDelPositioningMode mode) override {
|
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
|
|
|
(void)mode;
|
|
|
|
assert(mode == RangeDelPositioningMode::kFullScan);
|
|
|
|
for (const auto& tombstone : rep_) {
|
2018-10-10 00:15:27 +02:00
|
|
|
if (icmp_->Compare(parsed, tombstone.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
|
|
|
break;
|
|
|
|
}
|
|
|
|
if (parsed.sequence < tombstone.seq_ &&
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(parsed, tombstone.end_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
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2018-10-10 00:15:27 +02:00
|
|
|
bool IsRangeOverlapped(const ParsedInternalKey& start,
|
|
|
|
const ParsedInternalKey& end) override {
|
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
|
|
|
for (const auto& tombstone : rep_) {
|
2018-10-10 00:15:27 +02:00
|
|
|
if (icmp_->Compare(start, tombstone.end_key_) < 0 &&
|
|
|
|
icmp_->Compare(tombstone.start_key_, end) <= 0 &&
|
|
|
|
icmp_->Compare(tombstone.start_key_, tombstone.end_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
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2018-10-10 00:15:27 +02:00
|
|
|
void AddTombstone(TruncatedRangeTombstone tombstone) override {
|
2018-08-23 19:04:10 +02:00
|
|
|
rep_.emplace(tombstone);
|
|
|
|
}
|
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-08-15 00:03:57 +02:00
|
|
|
size_t Size() const override { return rep_.size(); }
|
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-08-15 00:03:57 +02:00
|
|
|
void InvalidatePosition() override {} // no-op
|
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-08-15 00:03:57 +02:00
|
|
|
std::unique_ptr<RangeDelIterator> NewIterator() override {
|
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
|
|
|
return std::unique_ptr<RangeDelIterator>(new Iterator(this->rep_));
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
// A CollapsedRangeDelMap is slow to create but quick to answer ShouldDelete
|
|
|
|
// queries.
|
|
|
|
//
|
|
|
|
// An explanation of the design follows. Suppose we have tombstones [b, n) @ 1,
|
|
|
|
// [e, h) @ 2, [q, t) @ 2, and [g, k) @ 3. Visually, the tombstones look like
|
|
|
|
// this:
|
|
|
|
//
|
|
|
|
// 3: g---k
|
|
|
|
// 2: e---h q--t
|
|
|
|
// 1: b------------n
|
|
|
|
//
|
|
|
|
// The CollapsedRangeDelMap representation is based on the observation that
|
|
|
|
// wherever tombstones overlap, we need only store the tombstone with the
|
|
|
|
// largest seqno. From the perspective of a read at seqno 4 or greater, this set
|
|
|
|
// of tombstones is exactly equivalent:
|
|
|
|
//
|
|
|
|
// 3: g---k
|
|
|
|
// 2: e--g q--t
|
|
|
|
// 1: b--e k--n
|
|
|
|
//
|
|
|
|
// Because these tombstones do not overlap, they can be efficiently represented
|
|
|
|
// in an ordered map from keys to sequence numbers. Each entry should be thought
|
|
|
|
// of as a transition from one tombstone to the next. In this example, the
|
|
|
|
// CollapsedRangeDelMap would store the following entries, in order:
|
|
|
|
//
|
|
|
|
// b → 1, e → 2, g → 3, k → 1, n → 0, q → 2, t → 0
|
|
|
|
//
|
|
|
|
// If a tombstone ends before the next tombstone begins, a sentinel seqno of 0
|
|
|
|
// is installed to indicate that no tombstone exists. This occurs at keys n and
|
|
|
|
// t in the example above.
|
|
|
|
//
|
|
|
|
// To check whether a key K is covered by a tombstone, the map is binary
|
|
|
|
// searched for the last key less than K. K is covered iff the map entry has a
|
|
|
|
// larger seqno than K. As an example, consider the key h @ 4. It would be
|
|
|
|
// compared against the map entry g → 3 and determined to be uncovered. By
|
|
|
|
// contrast, the key h @ 2 would be determined to be covered.
|
|
|
|
class CollapsedRangeDelMap : public RangeDelMap {
|
2018-10-10 00:15:27 +02:00
|
|
|
typedef std::map<ParsedInternalKey, SequenceNumber,
|
|
|
|
ParsedInternalKeyComparator>
|
|
|
|
Rep;
|
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
|
|
|
|
|
|
|
class Iterator : public RangeDelIterator {
|
|
|
|
void MaybeSeekPastSentinel() {
|
|
|
|
if (Valid() && iter_->second == 0) {
|
|
|
|
iter_++;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
const Rep& rep_;
|
|
|
|
Rep::const_iterator iter_;
|
|
|
|
|
|
|
|
public:
|
|
|
|
Iterator(const Rep& rep) : rep_(rep), iter_(rep.begin()) {}
|
|
|
|
|
|
|
|
bool Valid() const override { return iter_ != rep_.end(); }
|
|
|
|
|
|
|
|
void Next() override {
|
|
|
|
iter_++;
|
|
|
|
MaybeSeekPastSentinel();
|
|
|
|
}
|
|
|
|
|
2018-10-10 00:15:27 +02:00
|
|
|
void Seek(const Slice&) override {
|
|
|
|
fprintf(stderr, "CollapsedRangeDelMap::Iterator::Seek(Slice&) unimplemented\n");
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
|
|
|
void Seek(const ParsedInternalKey& target) override {
|
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
|
|
|
iter_ = rep_.upper_bound(target);
|
|
|
|
if (iter_ != rep_.begin()) {
|
|
|
|
iter_--;
|
|
|
|
}
|
|
|
|
MaybeSeekPastSentinel();
|
|
|
|
}
|
|
|
|
|
|
|
|
RangeTombstone Tombstone() const override {
|
2018-08-01 21:04:17 +02:00
|
|
|
assert(Valid());
|
|
|
|
assert(std::next(iter_) != rep_.end());
|
|
|
|
assert(iter_->second != 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
|
|
|
RangeTombstone tombstone;
|
2018-10-10 00:15:27 +02:00
|
|
|
tombstone.start_key_ = iter_->first.user_key;
|
|
|
|
tombstone.end_key_ = std::next(iter_)->first.user_key;
|
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
|
|
|
tombstone.seq_ = iter_->second;
|
|
|
|
return tombstone;
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
Rep rep_;
|
|
|
|
Rep::iterator iter_;
|
2018-10-10 00:15:27 +02:00
|
|
|
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
|
|
|
|
|
|
|
public:
|
2018-10-10 00:15:27 +02:00
|
|
|
explicit CollapsedRangeDelMap(const InternalKeyComparator* icmp)
|
|
|
|
: rep_(ParsedInternalKeyComparator(icmp)),
|
|
|
|
icmp_(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
|
|
|
InvalidatePosition();
|
|
|
|
}
|
|
|
|
|
|
|
|
bool ShouldDelete(const ParsedInternalKey& parsed,
|
2018-08-15 00:03:57 +02:00
|
|
|
RangeDelPositioningMode mode) override {
|
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
|
|
|
if (iter_ == rep_.end() &&
|
|
|
|
(mode == RangeDelPositioningMode::kForwardTraversal ||
|
|
|
|
mode == RangeDelPositioningMode::kBackwardTraversal)) {
|
|
|
|
// invalid (e.g., if AddTombstones() changed the deletions), so need to
|
|
|
|
// reseek
|
|
|
|
mode = RangeDelPositioningMode::kBinarySearch;
|
|
|
|
}
|
|
|
|
switch (mode) {
|
|
|
|
case RangeDelPositioningMode::kFullScan:
|
|
|
|
assert(false);
|
|
|
|
case RangeDelPositioningMode::kForwardTraversal:
|
|
|
|
assert(iter_ != rep_.end());
|
|
|
|
if (iter_ == rep_.begin() &&
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(parsed, iter_->first) < 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
|
|
|
// before start of deletion intervals
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
while (std::next(iter_) != rep_.end() &&
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(std::next(iter_)->first, parsed) <= 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
|
|
|
++iter_;
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
case RangeDelPositioningMode::kBackwardTraversal:
|
|
|
|
assert(iter_ != rep_.end());
|
|
|
|
while (iter_ != rep_.begin() &&
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(parsed, iter_->first) < 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
|
|
|
--iter_;
|
|
|
|
}
|
|
|
|
if (iter_ == rep_.begin() &&
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(parsed, iter_->first) < 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
|
|
|
// before start of deletion intervals
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
break;
|
|
|
|
case RangeDelPositioningMode::kBinarySearch:
|
2018-10-10 00:15:27 +02:00
|
|
|
iter_ = rep_.upper_bound(parsed);
|
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
|
|
|
if (iter_ == rep_.begin()) {
|
|
|
|
// before start of deletion intervals
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
--iter_;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
assert(iter_ != rep_.end() &&
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(iter_->first, parsed) <= 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
|
|
|
assert(std::next(iter_) == rep_.end() ||
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(parsed, std::next(iter_)->first) < 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
|
|
|
return parsed.sequence < iter_->second;
|
|
|
|
}
|
|
|
|
|
2018-10-10 00:15:27 +02:00
|
|
|
bool IsRangeOverlapped(const ParsedInternalKey&,
|
|
|
|
const ParsedInternalKey&) override {
|
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
|
|
|
// Unimplemented because the only client of this method, file ingestion,
|
|
|
|
// uses uncollapsed maps.
|
|
|
|
fprintf(stderr, "CollapsedRangeDelMap::IsRangeOverlapped unimplemented");
|
|
|
|
abort();
|
|
|
|
}
|
|
|
|
|
2018-10-10 00:15:27 +02:00
|
|
|
void AddTombstone(TruncatedRangeTombstone t) override {
|
|
|
|
if (icmp_->Compare(t.start_key_, t.end_key_) >= 0 || t.seq_ == 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
|
|
|
// The tombstone covers no keys. Nothing to do.
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
|
|
|
auto it = rep_.upper_bound(t.start_key_);
|
|
|
|
auto prev_seq = [&]() {
|
|
|
|
return it == rep_.begin() ? 0 : std::prev(it)->second;
|
|
|
|
};
|
|
|
|
|
|
|
|
// end_seq stores the seqno of the last transition that the new tombstone
|
|
|
|
// covered. This is the seqno that we'll install if we need to insert a
|
|
|
|
// transition for the new tombstone's end key.
|
|
|
|
SequenceNumber end_seq = 0;
|
|
|
|
|
|
|
|
// In the diagrams below, the new tombstone is always [c, k) @ 2. The
|
|
|
|
// existing tombstones are varied to depict different scenarios. Uppercase
|
|
|
|
// letters are used to indicate points that exist in the map, while
|
|
|
|
// lowercase letters are used to indicate points that do not exist in the
|
|
|
|
// map. The location of the iterator is marked with a caret; it may point
|
|
|
|
// off the end of the diagram to indicate that it is positioned at a
|
|
|
|
// entry with a larger key whose specific key is irrelevant.
|
|
|
|
|
|
|
|
if (t.seq_ > prev_seq()) {
|
|
|
|
// The new tombstone's start point covers the existing tombstone:
|
|
|
|
//
|
|
|
|
// 3: 3: A--C 3: 3:
|
|
|
|
// 2: c--- OR 2: c--- OR 2: c--- OR 2: c------
|
|
|
|
// 1: A--C 1: 1: A------ 1: C------
|
|
|
|
// ^ ^ ^ ^
|
|
|
|
end_seq = prev_seq();
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02:00
|
|
|
Rep::iterator pit;
|
|
|
|
if (it != rep_.begin() && (pit = std::prev(it)) != rep_.begin() &&
|
2018-10-10 00:15:27 +02:00
|
|
|
icmp_->Compare(pit->first, t.start_key_) == 0 &&
|
|
|
|
std::prev(pit)->second == t.seq_) {
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02:00
|
|
|
// The new tombstone starts at the end of an existing tombstone with an
|
|
|
|
// identical seqno:
|
|
|
|
//
|
|
|
|
// 3:
|
|
|
|
// 2: A--C---
|
|
|
|
// 1:
|
|
|
|
// ^
|
|
|
|
// Merge the tombstones by removing the existing tombstone's end key.
|
|
|
|
it = rep_.erase(std::prev(it));
|
|
|
|
} else {
|
|
|
|
// Insert a new transition at the new tombstone's start point, or raise
|
|
|
|
// the existing transition at that point to the new tombstone's seqno.
|
|
|
|
rep_[t.start_key_] = t.seq_; // operator[] will overwrite existing entry
|
|
|
|
}
|
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
|
|
|
} else {
|
|
|
|
// The new tombstone's start point is covered by an existing tombstone:
|
|
|
|
//
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02:00
|
|
|
// 3: A----- OR 3: C------ OR
|
|
|
|
// 2: c--- 2: c------ 2: C------
|
|
|
|
// ^ ^ ^
|
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
|
|
|
// Do nothing.
|
|
|
|
}
|
|
|
|
|
|
|
|
// Look at all the existing transitions that overlap the new tombstone.
|
2018-10-10 00:15:27 +02:00
|
|
|
while (it != rep_.end() && icmp_->Compare(it->first, t.end_key_) < 0) {
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02:00
|
|
|
if (t.seq_ >= it->second) {
|
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
|
|
|
// The transition is to an existing tombstone that the new tombstone
|
|
|
|
// covers. Save the covered tombstone's seqno. We'll need to return to
|
|
|
|
// it if the new tombstone ends before the existing tombstone.
|
|
|
|
end_seq = it->second;
|
|
|
|
|
|
|
|
if (t.seq_ == prev_seq()) {
|
|
|
|
// The previous transition is to the seqno of the new tombstone:
|
|
|
|
//
|
|
|
|
// 3: 3: 3: --F
|
|
|
|
// 2: C------ OR 2: C------ OR 2: F----
|
|
|
|
// 1: F--- 1: ---F 1: H--
|
|
|
|
// ^ ^ ^
|
|
|
|
//
|
|
|
|
// Erase this transition. It's been superseded.
|
|
|
|
it = rep_.erase(it);
|
|
|
|
continue; // skip increment; erase positions iterator correctly
|
|
|
|
} else {
|
|
|
|
// The previous transition is to a tombstone that covers the new
|
|
|
|
// tombstone, but this transition is to a tombstone that is covered by
|
|
|
|
// the new tombstone. That is, this is the end of a run of existing
|
|
|
|
// tombstones that cover the new tombstone:
|
|
|
|
//
|
|
|
|
// 3: A---E OR 3: E-G
|
|
|
|
// 2: c---- 2: ------
|
|
|
|
// ^ ^
|
|
|
|
// Preserve this transition point, but raise it to the new tombstone's
|
|
|
|
// seqno.
|
|
|
|
it->second = t.seq_;
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// The transition is to an existing tombstone that covers the new
|
|
|
|
// tombstone:
|
|
|
|
//
|
|
|
|
// 4: 4: --F
|
|
|
|
// 3: F-- OR 3: F--
|
|
|
|
// 2: ----- 2: -----
|
|
|
|
// ^ ^
|
|
|
|
// Do nothing.
|
|
|
|
}
|
|
|
|
++it;
|
|
|
|
}
|
|
|
|
|
|
|
|
if (t.seq_ == prev_seq()) {
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02:00
|
|
|
// The new tombstone is unterminated in the map.
|
2018-10-10 00:15:27 +02:00
|
|
|
if (it != rep_.end() && t.seq_ == it->second &&
|
|
|
|
icmp_->Compare(it->first, t.end_key_) == 0) {
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02:00
|
|
|
// The new tombstone ends at the start of another tombstone with an
|
|
|
|
// identical seqno. Merge the tombstones by removing the existing
|
|
|
|
// tombstone's start key.
|
|
|
|
rep_.erase(it);
|
2018-10-10 00:15:27 +02:00
|
|
|
} else if (end_seq == prev_seq() ||
|
|
|
|
(it != rep_.end() && end_seq == it->second)) {
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02:00
|
|
|
// The new tombstone is implicitly ended because its end point is
|
|
|
|
// contained within an existing tombstone with the same seqno:
|
|
|
|
//
|
|
|
|
// 2: ---k--N
|
|
|
|
// ^
|
|
|
|
} else {
|
|
|
|
// The new tombstone needs an explicit end point.
|
|
|
|
//
|
|
|
|
// 3: OR 3: --G OR 3: --G K--
|
|
|
|
// 2: C-------k 2: G---k 2: G---k
|
|
|
|
// ^ ^ ^
|
|
|
|
// Install one that returns to the last seqno we covered. Because end
|
|
|
|
// keys are exclusive, if there's an existing transition at t.end_key_,
|
|
|
|
// it takes precedence over the transition that we install here.
|
2018-10-10 00:15:27 +02:00
|
|
|
rep_.emplace(t.end_key_,
|
|
|
|
end_seq); // emplace is a noop if existing entry
|
Handle tombstones at the same seqno in the CollapsedRangeDelMap (#4424)
Summary:
The CollapsedRangeDelMap was entirely mishandling tombstones at the same
sequence number when the tombstones did not have identical start and end
keys. Such tombstones are common since 90fc40690, which causes
tombstones to be split during compactions.
For example, if the tombstone [a, c) @ 1 lies across a compaction
boundary at b, it will be split into [a, b) @ 1 and [b, c) @ 1. Without
this patch, the collapsed range deletion map would look like this:
a -> 1
b -> 1
c -> 0
Notice how the b -> 1 entry is redundant. When the tombstones overlap,
the problem is even worse. Consider tombstones [a, c) @ 1 and [b, d) @
1, which produces this map without this patch:
a -> 1
b -> 1
c -> 0
d -> 0
This map is corrupt, as a map can never contain adjacent sentinel (zero)
entries. When the iterator advances from b to c, it will notice that c
is a sentinel enty and skip to d--but d is also a sentinel entry! Asking
what tombstone this iterator points to will trigger an assertion, as it
is not pointing to a valid tombstone.
/cc ajkr
Pull Request resolved: https://github.com/facebook/rocksdb/pull/4424
Differential Revision: D10039248
Pulled By: abhimadan
fbshipit-source-id: 6d737c1e88d60e80cf27286726627ba44463e7f4
2018-09-25 23:48:44 +02: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
|
|
|
} else {
|
|
|
|
// The new tombstone is implicitly ended because its end point is covered
|
|
|
|
// by an existing tombstone with a higher seqno.
|
|
|
|
//
|
|
|
|
// 3: I---M OR 3: A-----------M
|
|
|
|
// 2: ----k 2: c-------k
|
|
|
|
// ^ ^
|
|
|
|
// Do nothing.
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-08-31 21:18:11 +02:00
|
|
|
size_t Size() const override { return rep_.empty() ? 0 : rep_.size() - 1; }
|
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-08-15 00:03:57 +02:00
|
|
|
void InvalidatePosition() override { iter_ = rep_.end(); }
|
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-08-15 00:03:57 +02:00
|
|
|
std::unique_ptr<RangeDelIterator> NewIterator() override {
|
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
|
|
|
return std::unique_ptr<RangeDelIterator>(new Iterator(this->rep_));
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
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
|
|
|
RangeDelAggregator::RangeDelAggregator(
|
|
|
|
const InternalKeyComparator& icmp,
|
2016-12-20 01:44:30 +01:00
|
|
|
const std::vector<SequenceNumber>& snapshots,
|
|
|
|
bool collapse_deletions /* = true */)
|
|
|
|
: upper_bound_(kMaxSequenceNumber),
|
|
|
|
icmp_(icmp),
|
|
|
|
collapse_deletions_(collapse_deletions) {
|
2016-11-19 01:54:09 +01:00
|
|
|
InitRep(snapshots);
|
|
|
|
}
|
|
|
|
|
|
|
|
RangeDelAggregator::RangeDelAggregator(const InternalKeyComparator& icmp,
|
2016-12-20 01:44:30 +01:00
|
|
|
SequenceNumber snapshot,
|
|
|
|
bool collapse_deletions /* = false */)
|
|
|
|
: upper_bound_(snapshot),
|
|
|
|
icmp_(icmp),
|
|
|
|
collapse_deletions_(collapse_deletions) {}
|
2016-11-19 01:54:09 +01:00
|
|
|
|
|
|
|
void RangeDelAggregator::InitRep(const std::vector<SequenceNumber>& snapshots) {
|
|
|
|
assert(rep_ == nullptr);
|
|
|
|
rep_.reset(new Rep());
|
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
|
|
|
for (auto snapshot : snapshots) {
|
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
|
|
|
rep_->stripe_map_.emplace(snapshot, NewRangeDelMap());
|
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
|
|
|
}
|
|
|
|
// Data newer than any snapshot falls in this catch-all stripe
|
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
|
|
|
rep_->stripe_map_.emplace(kMaxSequenceNumber, NewRangeDelMap());
|
2016-11-19 01:54:09 +01:00
|
|
|
rep_->pinned_iters_mgr_.StartPinning();
|
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
|
|
|
}
|
|
|
|
|
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
|
|
|
std::unique_ptr<RangeDelMap> RangeDelAggregator::NewRangeDelMap() {
|
|
|
|
RangeDelMap* tombstone_map;
|
|
|
|
if (collapse_deletions_) {
|
2018-10-10 00:15:27 +02:00
|
|
|
tombstone_map = new CollapsedRangeDelMap(&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
|
|
|
} else {
|
2018-10-10 00:15:27 +02:00
|
|
|
tombstone_map = new UncollapsedRangeDelMap(&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
|
|
|
}
|
|
|
|
return std::unique_ptr<RangeDelMap>(tombstone_map);
|
|
|
|
}
|
|
|
|
|
|
|
|
bool RangeDelAggregator::ShouldDeleteImpl(const Slice& internal_key,
|
|
|
|
RangeDelPositioningMode mode) {
|
2017-09-15 00:41:19 +02:00
|
|
|
assert(rep_ != nullptr);
|
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
|
|
|
ParsedInternalKey parsed;
|
|
|
|
if (!ParseInternalKey(internal_key, &parsed)) {
|
|
|
|
assert(false);
|
2018-10-10 00:15:27 +02:00
|
|
|
return false;
|
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
|
|
|
}
|
2018-10-10 00:15:27 +02:00
|
|
|
return ShouldDeleteImpl(parsed, mode);
|
2016-11-04 02:40:23 +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
|
|
|
bool RangeDelAggregator::ShouldDeleteImpl(const ParsedInternalKey& parsed,
|
|
|
|
RangeDelPositioningMode mode) {
|
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
|
|
|
assert(IsValueType(parsed.type));
|
2017-09-15 00:41:19 +02:00
|
|
|
assert(rep_ != nullptr);
|
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
|
|
|
auto& tombstone_map = GetRangeDelMap(parsed.sequence);
|
|
|
|
if (tombstone_map.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
|
|
|
return false;
|
2016-12-20 01:44:30 +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
|
|
|
return tombstone_map.ShouldDelete(parsed, mode);
|
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
|
|
|
}
|
|
|
|
|
2017-11-28 20:18:42 +01:00
|
|
|
bool RangeDelAggregator::IsRangeOverlapped(const Slice& start,
|
|
|
|
const Slice& end) {
|
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
|
|
|
// Unimplemented because the only client of this method, file ingestion,
|
|
|
|
// uses uncollapsed maps.
|
2017-11-28 20:18:42 +01:00
|
|
|
assert(!collapse_deletions_);
|
|
|
|
if (rep_ == nullptr) {
|
|
|
|
return false;
|
|
|
|
}
|
2018-10-10 00:15:27 +02:00
|
|
|
ParsedInternalKey start_ikey(start, kMaxSequenceNumber, kMaxValue);
|
|
|
|
ParsedInternalKey end_ikey(end, 0, static_cast<ValueType>(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
|
|
|
for (const auto& stripe : rep_->stripe_map_) {
|
2018-10-10 00:15:27 +02:00
|
|
|
if (stripe.second->IsRangeOverlapped(start_ikey, end_ikey)) {
|
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
|
|
|
return true;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2016-10-19 19:59:46 +02:00
|
|
|
Status RangeDelAggregator::AddTombstones(
|
2018-07-14 02:34:54 +02:00
|
|
|
std::unique_ptr<InternalIterator> input,
|
|
|
|
const InternalKey* smallest,
|
|
|
|
const InternalKey* largest) {
|
2016-11-19 23:14:35 +01:00
|
|
|
if (input == nullptr) {
|
|
|
|
return Status::OK();
|
|
|
|
}
|
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
|
|
|
input->SeekToFirst();
|
2016-11-19 01:54:09 +01:00
|
|
|
bool first_iter = true;
|
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
|
|
|
while (input->Valid()) {
|
2016-11-19 01:54:09 +01:00
|
|
|
if (first_iter) {
|
|
|
|
if (rep_ == nullptr) {
|
|
|
|
InitRep({upper_bound_});
|
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
|
|
|
} else {
|
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
|
|
|
InvalidateRangeDelMapPositions();
|
2016-11-19 01:54:09 +01:00
|
|
|
}
|
|
|
|
first_iter = false;
|
|
|
|
}
|
2016-10-19 19:59:46 +02:00
|
|
|
ParsedInternalKey parsed_key;
|
2018-07-14 07:40:23 +02:00
|
|
|
bool parsed;
|
|
|
|
if (input->IsKeyPinned()) {
|
|
|
|
parsed = ParseInternalKey(input->key(), &parsed_key);
|
|
|
|
} else {
|
|
|
|
// The tombstone map holds slices into the iterator's memory. Make a
|
|
|
|
// copy of the key if it is not pinned.
|
|
|
|
rep_->pinned_slices_.emplace_back(input->key().data(),
|
|
|
|
input->key().size());
|
|
|
|
parsed = ParseInternalKey(rep_->pinned_slices_.back(), &parsed_key);
|
|
|
|
}
|
|
|
|
if (!parsed) {
|
2016-10-19 19:59:46 +02:00
|
|
|
return Status::Corruption("Unable to parse range tombstone InternalKey");
|
|
|
|
}
|
2018-10-10 00:15:27 +02:00
|
|
|
Slice end_user_key;
|
2018-07-14 07:40:23 +02:00
|
|
|
if (input->IsValuePinned()) {
|
2018-10-10 00:15:27 +02:00
|
|
|
end_user_key = input->value();
|
2018-07-14 07:40:23 +02:00
|
|
|
} else {
|
|
|
|
// The tombstone map holds slices into the iterator's memory. Make a
|
|
|
|
// copy of the value if it is not pinned.
|
|
|
|
rep_->pinned_slices_.emplace_back(input->value().data(),
|
|
|
|
input->value().size());
|
2018-10-10 00:15:27 +02:00
|
|
|
end_user_key = rep_->pinned_slices_.back();
|
2018-07-14 07:40:23 +02:00
|
|
|
}
|
2018-10-10 00:15:27 +02:00
|
|
|
ParsedInternalKey start_key(parsed_key.user_key, kMaxSequenceNumber,
|
|
|
|
kMaxValue);
|
|
|
|
ParsedInternalKey end_key(end_user_key, kMaxSequenceNumber, kMaxValue);
|
2018-07-14 02:34:54 +02:00
|
|
|
// Truncate the tombstone to the range [smallest, largest].
|
|
|
|
if (smallest != nullptr) {
|
2018-10-10 00:15:27 +02:00
|
|
|
ParsedInternalKey parsed_smallest;
|
|
|
|
if (ParseInternalKey(smallest->Encode(), &parsed_smallest) &&
|
|
|
|
icmp_.Compare(start_key, parsed_smallest) < 0) {
|
|
|
|
start_key.user_key = parsed_smallest.user_key;
|
|
|
|
start_key.sequence = parsed_smallest.sequence;
|
2018-07-14 02:34:54 +02:00
|
|
|
}
|
|
|
|
}
|
|
|
|
if (largest != nullptr) {
|
2018-10-10 00:15:27 +02:00
|
|
|
ParsedInternalKey parsed_largest;
|
|
|
|
if (ParseInternalKey(largest->Encode(), &parsed_largest) &&
|
|
|
|
icmp_.Compare(end_key, parsed_largest) > 0) {
|
|
|
|
end_key.user_key = parsed_largest.user_key;
|
|
|
|
if (parsed_largest.sequence != kMaxSequenceNumber) {
|
|
|
|
// The same user key straddles two adjacent sstables. To make sure we
|
|
|
|
// can truncate to a range that includes the largest point key in the
|
|
|
|
// first sstable, set the tombstone end key's sequence number to 1
|
|
|
|
// less than the largest key.
|
|
|
|
assert(parsed_largest.sequence != 0);
|
|
|
|
end_key.sequence = parsed_largest.sequence - 1;
|
|
|
|
} else {
|
|
|
|
// The SST file boundary was artificially extended by a range tombstone.
|
|
|
|
// We will not see any entries in this SST with this user key, so we
|
|
|
|
// can leave the seqnum at kMaxSequenceNumber.
|
|
|
|
}
|
2018-07-14 02:34:54 +02:00
|
|
|
}
|
|
|
|
}
|
2018-10-10 00:15:27 +02:00
|
|
|
TruncatedRangeTombstone tombstone(start_key, end_key, parsed_key.sequence);
|
|
|
|
GetRangeDelMap(parsed_key.sequence).AddTombstone(std::move(tombstone));
|
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
|
|
|
input->Next();
|
|
|
|
}
|
2016-11-19 01:54:09 +01:00
|
|
|
if (!first_iter) {
|
2016-11-19 23:14:35 +01:00
|
|
|
rep_->pinned_iters_mgr_.PinIterator(input.release(), false /* arena */);
|
2016-11-19 01:54:09 +01:00
|
|
|
}
|
2016-10-19 19:59:46 +02:00
|
|
|
return Status::OK();
|
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
|
|
|
}
|
|
|
|
|
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
|
|
|
void RangeDelAggregator::InvalidateRangeDelMapPositions() {
|
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
|
|
|
if (rep_ == nullptr) {
|
|
|
|
return;
|
|
|
|
}
|
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
|
|
|
for (auto& stripe : rep_->stripe_map_) {
|
|
|
|
stripe.second->InvalidatePosition();
|
2016-12-20 01:44:30 +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
|
|
|
RangeDelMap& RangeDelAggregator::GetRangeDelMap(SequenceNumber seq) {
|
2016-11-19 01:54:09 +01:00
|
|
|
assert(rep_ != nullptr);
|
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
|
|
|
// The stripe includes seqnum for the snapshot above and excludes seqnum for
|
|
|
|
// the snapshot below.
|
|
|
|
StripeMap::iterator iter;
|
|
|
|
if (seq > 0) {
|
|
|
|
// upper_bound() checks strict inequality so need to subtract one
|
2016-11-19 01:54:09 +01:00
|
|
|
iter = rep_->stripe_map_.upper_bound(seq - 1);
|
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
|
|
|
} else {
|
2016-11-19 01:54:09 +01:00
|
|
|
iter = rep_->stripe_map_.begin();
|
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
|
|
|
}
|
|
|
|
// catch-all stripe justifies this assertion in either of above cases
|
2016-11-19 01:54:09 +01:00
|
|
|
assert(iter != rep_->stripe_map_.end());
|
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
|
|
|
return *iter->second;
|
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-04 19:53:38 +01:00
|
|
|
bool RangeDelAggregator::IsEmpty() {
|
2016-11-19 01:54:09 +01:00
|
|
|
if (rep_ == nullptr) {
|
|
|
|
return true;
|
|
|
|
}
|
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
|
|
|
for (const auto& stripe : rep_->stripe_map_) {
|
|
|
|
if (!stripe.second->IsEmpty()) {
|
2016-11-04 19:53:38 +01:00
|
|
|
return false;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2018-05-05 01:37:39 +02:00
|
|
|
bool RangeDelAggregator::AddFile(uint64_t file_number) {
|
|
|
|
if (rep_ == nullptr) {
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
return rep_->added_files_.emplace(file_number).second;
|
|
|
|
}
|
|
|
|
|
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
|
|
|
class MergingRangeDelIter : public RangeDelIterator {
|
|
|
|
public:
|
|
|
|
MergingRangeDelIter(const Comparator* c)
|
|
|
|
: heap_(IterMinHeap(IterComparator(c))), current_(nullptr) {}
|
|
|
|
|
|
|
|
void AddIterator(std::unique_ptr<RangeDelIterator> iter) {
|
|
|
|
if (iter->Valid()) {
|
|
|
|
heap_.push(iter.get());
|
|
|
|
iters_.push_back(std::move(iter));
|
|
|
|
current_ = heap_.top();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
bool Valid() const override { return current_ != nullptr; }
|
|
|
|
|
|
|
|
void Next() override {
|
|
|
|
current_->Next();
|
|
|
|
if (current_->Valid()) {
|
|
|
|
heap_.replace_top(current_);
|
|
|
|
} else {
|
|
|
|
heap_.pop();
|
|
|
|
}
|
|
|
|
current_ = heap_.empty() ? nullptr : heap_.top();
|
|
|
|
}
|
|
|
|
|
|
|
|
void Seek(const Slice& target) override {
|
2018-10-10 00:15:27 +02:00
|
|
|
ParsedInternalKey ikey(target, kMaxSequenceNumber, kMaxValue);
|
|
|
|
Seek(ikey);
|
|
|
|
}
|
|
|
|
|
|
|
|
void Seek(const ParsedInternalKey& target) override {
|
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
|
|
|
heap_.clear();
|
|
|
|
for (auto& iter : iters_) {
|
|
|
|
iter->Seek(target);
|
|
|
|
if (iter->Valid()) {
|
|
|
|
heap_.push(iter.get());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
current_ = heap_.empty() ? nullptr : heap_.top();
|
|
|
|
}
|
|
|
|
|
|
|
|
RangeTombstone Tombstone() const override { return current_->Tombstone(); }
|
|
|
|
|
|
|
|
private:
|
|
|
|
struct IterComparator {
|
|
|
|
IterComparator(const Comparator* c) : cmp(c) {}
|
|
|
|
|
|
|
|
bool operator()(const RangeDelIterator* a,
|
|
|
|
const RangeDelIterator* b) const {
|
|
|
|
// Note: counterintuitively, returning the tombstone with the larger start
|
|
|
|
// key puts the tombstone with the smallest key at the top of the heap.
|
|
|
|
return cmp->Compare(a->Tombstone().start_key_,
|
|
|
|
b->Tombstone().start_key_) > 0;
|
|
|
|
}
|
|
|
|
|
|
|
|
const Comparator* cmp;
|
|
|
|
};
|
|
|
|
|
|
|
|
typedef BinaryHeap<RangeDelIterator*, IterComparator> IterMinHeap;
|
|
|
|
|
|
|
|
std::vector<std::unique_ptr<RangeDelIterator>> iters_;
|
|
|
|
IterMinHeap heap_;
|
|
|
|
RangeDelIterator* current_;
|
|
|
|
};
|
|
|
|
|
|
|
|
std::unique_ptr<RangeDelIterator> RangeDelAggregator::NewIterator() {
|
|
|
|
std::unique_ptr<MergingRangeDelIter> iter(
|
|
|
|
new MergingRangeDelIter(icmp_.user_comparator()));
|
|
|
|
if (rep_ != nullptr) {
|
|
|
|
for (const auto& stripe : rep_->stripe_map_) {
|
|
|
|
iter->AddIterator(stripe.second->NewIterator());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return std::move(iter);
|
|
|
|
}
|
|
|
|
|
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
|
|
|
} // namespace rocksdb
|