diff --git a/level_checker.go b/level_checker.go index 7da59c9e81..b378f1c3df 100644 --- a/level_checker.go +++ b/level_checker.go @@ -202,24 +202,17 @@ func (m *simpleMergingIter) step() bool { return false } // Is this point covered by a tombstone at a lower level? Note that all these - // iterators must be positioned at a key > item.key. So the Largest key bound - // of the sstable containing the tombstone >= item.key. So the upper limit of - // the tombstone cannot be file-bounds-constrained to < item.key. But it is - // possible that item.key < smallest key bound of the sstable, in which case - // this tombstone should be ignored. + // iterators must be positioned at a key > item.key. for level := item.index + 1; level < len(m.levels); level++ { lvl := &m.levels[level] if lvl.rangeDelIter == nil || lvl.tombstone.Empty() { continue } - if (lvl.smallestUserKey == nil || m.heap.cmp(lvl.smallestUserKey, item.key.UserKey) <= 0) && - lvl.tombstone.Contains(m.heap.cmp, item.key.UserKey) { - if lvl.tombstone.CoversAt(m.snapshot, item.key.SeqNum()) { - m.err = errors.Errorf("tombstone %s in %s deletes key %s in %s", - lvl.tombstone.Pretty(m.formatKey), lvl.iter, item.key.Pretty(m.formatKey), - l.iter) - return false - } + if lvl.tombstone.Contains(m.heap.cmp, item.key.UserKey) && lvl.tombstone.CoversAt(m.snapshot, item.key.SeqNum()) { + m.err = errors.Errorf("tombstone %s in %s deletes key %s in %s", + lvl.tombstone.Pretty(m.formatKey), lvl.iter, item.key.Pretty(m.formatKey), + l.iter) + return false } } } diff --git a/level_iter.go b/level_iter.go index 000ca483ae..e738b22f2d 100644 --- a/level_iter.go +++ b/level_iter.go @@ -696,11 +696,6 @@ func (l *levelIter) loadFile(file *fileMetadata, dir int) loadFileReturnIndicato } else if rangeDelIter != nil { rangeDelIter.Close() } - if l.boundaryContext != nil { - l.boundaryContext.smallestUserKey = file.Smallest.UserKey - l.boundaryContext.largestUserKey = file.Largest.UserKey - l.boundaryContext.isLargestUserKeyExclusive = file.Largest.IsExclusiveSentinel() - } return newFileLoaded } } diff --git a/merging_iter.go b/merging_iter.go index ce08647d88..d1028eb95b 100644 --- a/merging_iter.go +++ b/merging_iter.go @@ -48,14 +48,6 @@ type mergingIterLevel struct { } type levelIterBoundaryContext struct { - // smallestUserKey and largestUserKey are populated with the smallest and - // largest boundaries of the current file. - smallestUserKey, largestUserKey []byte - // isLargestUserKeyExclusive is set to true when a file's largest boundary - // is an exclusive key, (eg, a range deletion sentinel). If true, the file - // does not contain any keys with the provided user key, and the - // largestUserKey bound is exclusive. - isLargestUserKeyExclusive bool // isSyntheticIterBoundsKey is set to true iff the key returned by the level // iterator is a synthetic key derived from the iterator bounds. This is used // to prevent the mergingIter from being stuck at such a synthetic key if it @@ -737,39 +729,13 @@ func (m *mergingIter) isNextEntryDeleted(item *mergingIterLevel) (bool, error) { continue } - // Reasoning for correctness of untruncated tombstone handling when the untruncated - // tombstone is at a higher level: - // The iterator corresponding to this tombstone is still in the heap so it must be - // positioned >= item.iterKey. Which means the Largest key bound of the sstable containing this - // tombstone is >= item.iterKey. So the upper limit of this tombstone cannot be file-bounds-constrained - // to < item.iterKey. But it is possible that item.key < smallestUserKey, in which - // case this tombstone should be ignored. - // - // Example 1: - // sstable bounds [c#8, g#12] containing a tombstone [b, i)#7, and key is c#6. The - // smallestUserKey is c, so we know the key is within the file bounds and the tombstone - // [b, i) covers it. - // - // Example 2: - // Same sstable bounds but key is b#10. The smallestUserKey is c, so the tombstone [b, i) - // does not cover this key. - // - // For a tombstone at the same level as the key, the file bounds are trivially satisfied. - if (l.smallestUserKey == nil || m.heap.cmp(l.smallestUserKey, item.iterKey.UserKey) <= 0) && - l.tombstone.VisibleAt(m.snapshot) && l.tombstone.Contains(m.heap.cmp, item.iterKey.UserKey) { + if l.tombstone.VisibleAt(m.snapshot) && l.tombstone.Contains(m.heap.cmp, item.iterKey.UserKey) { if level < item.index { // We could also do m.seekGE(..., level + 1). The levels from // [level + 1, item.index) are already after item.iterKey so seeking them may be // wasteful. - // We can seek up to the min of largestUserKey and tombstone.End. - // - // Using example 1 above, we can seek to the smaller of g and i, which is g. - // - // Another example, where the sstable bounds are [c#8, i#InternalRangeDelSentinel], - // and the tombstone is [b, i)#8. Seeking to i is correct since it is seeking up to - // the exclusive bound of the tombstone. We do not need to look at - // isLargestKeyRangeDelSentinel. + // We can seek up to tombstone.End. // // Progress argument: Since this file is at a higher level than item.iterKey we know // that the iterator in this file must be positioned within its bounds and at a key @@ -781,9 +747,6 @@ func (m *mergingIter) isNextEntryDeleted(item *mergingIterLevel) (bool, error) { // seekKey, computed below, is > item.iterKey.UserKey, so the call to seekGE() will // make forward progress. seekKey := l.tombstone.End - if l.largestUserKey != nil && m.heap.cmp(l.largestUserKey, seekKey) < 0 { - seekKey = l.largestUserKey - } // This seek is not directly due to a SeekGE call, so we don't know // enough about the underlying iterator positions, and so we keep the // try-seek-using-next optimization disabled. Additionally, if we're in @@ -974,49 +937,13 @@ func (m *mergingIter) isPrevEntryDeleted(item *mergingIterLevel) (bool, error) { if l.tombstone == nil { continue } - - // Reasoning for correctness of untruncated tombstone handling when the untruncated - // tombstone is at a higher level: - // - // The iterator corresponding to this tombstone is still in the heap so it must be - // positioned <= item.iterKey. Which means the Smallest key bound of the sstable containing this - // tombstone is <= item.iterKey. So the lower limit of this tombstone cannot have been - // file-bounds-constrained to > item.iterKey. But it is possible that item.key >= Largest - // key bound of this sstable, in which case this tombstone should be ignored. - // - // Example 1: - // sstable bounds [c#8, g#12] containing a tombstone [b, i)#7, and key is f#6. The - // largestUserKey is g, so we know the key is within the file bounds and the tombstone - // [b, i) covers it. - // - // Example 2: - // Same sstable but the key is g#6. This cannot happen since the [b, i)#7 untruncated - // tombstone was involved in a compaction which must have had a file to the right of this - // sstable that is part of the same atomic compaction group for future compactions. That - // file must have bounds that cover g#6 and this levelIter must be at that file. - // - // Example 3: - // sstable bounds [c#8, g#RangeDelSentinel] containing [b, i)#7 and the key is g#10. - // This key is not deleted by this tombstone. We need to look at - // isLargestUserKeyExclusive. - // - // For a tombstone at the same level as the key, the file bounds are trivially satisfied. - - // Default to within bounds. - withinLargestSSTableBound := true - if l.largestUserKey != nil { - cmpResult := m.heap.cmp(l.largestUserKey, item.iterKey.UserKey) - withinLargestSSTableBound = cmpResult > 0 || (cmpResult == 0 && !l.isLargestUserKeyExclusive) - } - if withinLargestSSTableBound && l.tombstone.Contains(m.heap.cmp, item.iterKey.UserKey) && l.tombstone.VisibleAt(m.snapshot) { + if l.tombstone.Contains(m.heap.cmp, item.iterKey.UserKey) && l.tombstone.VisibleAt(m.snapshot) { if level < item.index { // We could also do m.seekLT(..., level + 1). The levels from // [level + 1, item.index) are already before item.iterKey so seeking them may be // wasteful. - // We can seek up to the max of smallestUserKey and tombstone.Start.UserKey. - // - // Using example 1 above, we can seek to the larger of c and b, which is c. + // We can seek up to tombstone.Start.UserKey. // // Progress argument: We know that the iterator in this file is positioned within // its bounds and at a key X < item.iterKey (otherwise it would be the max of the heap). @@ -1025,9 +952,6 @@ func (m *mergingIter) isPrevEntryDeleted(item *mergingIterLevel) (bool, error) { // is <= item.iterKey.UserKey, and since we do a seekLT() we will make backwards // progress. seekKey := l.tombstone.Start - if l.smallestUserKey != nil && m.heap.cmp(l.smallestUserKey, seekKey) > 0 { - seekKey = l.smallestUserKey - } // We set the relative-seek flag. This is important when // iterating with lazy combined iteration. If there's a range // key between this level's current file and the file the seek @@ -1159,42 +1083,21 @@ func (m *mergingIter) seekGE(key []byte, level int, flags base.SeekGEFlags) erro (m.combinedIterState == nil || m.combinedIterState.initialized) { // The level has a range-del iterator. Find the tombstone containing // the search key. - // - // For untruncated tombstones that are possibly file-bounds-constrained, we are using a - // levelIter which will set smallestUserKey and largestUserKey. Since the levelIter - // is at this file we know that largestUserKey >= key, so we know that the - // tombstone we find cannot be file-bounds-constrained in its upper bound to something < key. - // We do need to compare with smallestUserKey to ensure that the tombstone is not - // file-bounds-constrained in its lower bound. - // - // See the detailed comments in isNextEntryDeleted() on why similar containment and - // seeking logic is correct. The subtle difference here is that key is a user key, - // so we can have a sstable with bounds [c#8, i#InternalRangeDelSentinel], and the - // tombstone is [b, k)#8 and the seek key is i: levelIter.SeekGE(i) will move past - // this sstable since it realizes the largest key is a InternalRangeDelSentinel. var err error l.tombstone, err = rangeDelIter.SeekGE(key) if err != nil { return err } - if l.tombstone != nil && l.tombstone.VisibleAt(m.snapshot) && l.tombstone.Contains(m.heap.cmp, key) && - (l.smallestUserKey == nil || m.heap.cmp(l.smallestUserKey, key) <= 0) { - // NB: Based on the comment above l.largestUserKey >= key, and based on the - // containment condition tombstone.End > key, so the assignment to key results - // in a monotonically non-decreasing key across iterations of this loop. + if l.tombstone != nil && l.tombstone.VisibleAt(m.snapshot) && l.tombstone.Contains(m.heap.cmp, key) { + // Based on the containment condition tombstone.End > key, so + // the assignment to key results in a monotonically + // non-decreasing key across iterations of this loop. // - // The adjustment of key here can only move it to a larger key. Since - // the caller of seekGE guaranteed that the original key was greater - // than or equal to m.lower, the new key will continue to be greater - // than or equal to m.lower. - if l.largestUserKey != nil && - m.heap.cmp(l.largestUserKey, l.tombstone.End) < 0 { - // Truncate the tombstone for seeking purposes. Note that this can over-truncate - // but that is harmless for this seek optimization. - key = l.largestUserKey - } else { - key = l.tombstone.End - } + // The adjustment of key here can only move it to a larger key. + // Since the caller of seekGE guaranteed that the original key + // was greater than or equal to m.lower, the new key will + // continue to be greater than or equal to m.lower. + key = l.tombstone.End } } } @@ -1259,48 +1162,23 @@ func (m *mergingIter) seekLT(key []byte, level int, flags base.SeekLTFlags) erro (m.combinedIterState == nil || m.combinedIterState.initialized) { // The level has a range-del iterator. Find the tombstone containing // the search key. - // - // For untruncated tombstones that are possibly file-bounds-constrained we are using a - // levelIter which will set smallestUserKey and largestUserKey. Since the levelIter - // is at this file we know that smallestUserKey <= key, so we know that the - // tombstone we find cannot be file-bounds-constrained in its lower bound to something > key. - // We do need to compare with largestUserKey to ensure that the tombstone is not - // file-bounds-constrained in its upper bound. - // - // See the detailed comments in isPrevEntryDeleted() on why similar containment and - // seeking logic is correct. - - // Default to within bounds. - withinLargestSSTableBound := true - if l.largestUserKey != nil { - cmpResult := m.heap.cmp(l.largestUserKey, key) - withinLargestSSTableBound = cmpResult > 0 || (cmpResult == 0 && !l.isLargestUserKeyExclusive) - } - tomb, err := keyspan.SeekLE(m.heap.cmp, rangeDelIter, key) if err != nil { return err } l.tombstone = tomb if l.tombstone != nil && l.tombstone.VisibleAt(m.snapshot) && - l.tombstone.Contains(m.heap.cmp, key) && withinLargestSSTableBound { - // NB: Based on the comment above l.smallestUserKey <= key, and based - // on the containment condition tombstone.Start.UserKey <= key, so the - // assignment to key results in a monotonically non-increasing key - // across iterations of this loop. + l.tombstone.Contains(m.heap.cmp, key) { + // NB: Based on the containment condition + // tombstone.Start.UserKey <= key, so the assignment to key + // results in a monotonically non-increasing key across + // iterations of this loop. // - // The adjustment of key here can only move it to a smaller key. Since - // the caller of seekLT guaranteed that the original key was less than - // or equal to m.upper, the new key will continue to be less than or - // equal to m.upper. - if l.smallestUserKey != nil && - m.heap.cmp(l.smallestUserKey, l.tombstone.Start) >= 0 { - // Truncate the tombstone for seeking purposes. Note that this can over-truncate - // but that is harmless for this seek optimization. - key = l.smallestUserKey - } else { - key = l.tombstone.Start - } + // The adjustment of key here can only move it to a smaller key. + // Since the caller of seekLT guaranteed that the original key + // was less than or equal to m.upper, the new key will continue + // to be less than or equal to m.upper. + key = l.tombstone.Start } } } diff --git a/table_stats.go b/table_stats.go index f2afbd848c..7e8e8b6d80 100644 --- a/table_stats.go +++ b/table_stats.go @@ -910,23 +910,52 @@ func newCombinedDeletionKeyspanIter( return nil, err } if iter != nil { - // Assert expected bounds in tests. - if invariants.Enabled { - // TODO(radu): we should be using AssertBounds, but it currently fails in - // some cases (#3167). - iter = keyspan.AssertUserKeyBounds( - iter, m.SmallestPointKey.UserKey, m.LargestPointKey.UserKey, comparer.Compare, - ) - } + // Assert expected bounds. In previous versions of Pebble, range + // deletions persisted to sstables could exceed the bounds of the + // containing files due to "split user keys." This required readers to + // constrain the tombstones' bounds to the containing file at read time. + // See docs/range_deletions.md for an extended discussion of the design + // and invariants at that time. + // + // We've since compacted away all 'split user-keys' and in the process + // eliminated all "untruncated range tombstones" for physical sstables. + // We no longer need to perform truncation at read time for these + // sstables. + // + // At the same time, we've also introduced the concept of "virtual + // SSTables" where the file metadata's effective bounds can again be + // reduced to be narrower than the contained tombstones. These virtual + // SSTables handle truncation differently, performing it using + // keyspan.Truncate when the sstable's range deletion iterator is + // opened. + // + // Together, these mean that we should never see untruncated range + // tombstones any moreā€”and the merging iterator no longer accounts for + // their existence. Since there's abundant subtlety that we're relying + // on, we choose to be conservative and assert that these invariants + // hold. We could (and previously did) choose to only validate these + // bounds in invariants builds, but the most likely avenue for these + // tombstones' existence is through a bug in a migration and old data + // sitting around in an old store from long ago. + // + // The table stats collector will read all files range deletions + // asynchronously after Open, and provides a perfect opportunity to + // validate our invariants without harming user latency. We also + // previously performed truncation here which similarly required key + // comparisons, so replacing those key comparisons with assertions + // should be roughly similar in performance. + // + // TODO(jackson): Only use Assert[UserKey]Bounds in invariants builds + // in the following release. + // + // TODO(radu): we should be using AssertBounds, but it currently fails in + // some cases (#3167). + iter = keyspan.AssertUserKeyBounds( + iter, m.SmallestPointKey.UserKey, m.LargestPointKey.UserKey, comparer.Compare, + ) dIter := &keyspan.DefragmentingIter{} dIter.Init(comparer, iter, equal, reducer, new(keyspan.DefragmentingBuffers)) iter = dIter - // Truncate tombstones to the containing file's bounds if necessary. - // See docs/range_deletions.md for why this is necessary. - iter = keyspan.Truncate( - comparer.Compare, iter, m.Smallest.UserKey, m.Largest.UserKey, - nil, nil, false, /* panicOnUpperTruncate */ - ) mIter.AddLevel(iter) } diff --git a/testdata/merging_iter b/testdata/merging_iter index 89ade7a143..928f13ece1 100644 --- a/testdata/merging_iter +++ b/testdata/merging_iter @@ -13,10 +13,10 @@ define L a.SET.30 e.RANGEDEL.72057594037927935 -a.SET.30:30 c.SET.27:27 a.RANGEDEL.8:f +a.SET.30:30 c.SET.27:27 a.RANGEDEL.8:e L e.SET.10 g.SET.20 -e.SET.10:10 g.SET.20:20 e.RANGEDEL.8:f +e.SET.10:10 g.SET.20:20 e.RANGEDEL.8:g ---- 1: 000000:[a#30,SET-e#inf,RANGEDEL] @@ -57,11 +57,11 @@ iter probe-rangedels=(000000,(Log "# 000000.rangeDelIter.")) probe-rangedels=(0 seek-ge d next ---- -# 000000.rangeDelIter.opSpanSeekGE("d") = a-f:{(#8,RANGEDEL)} -# 000001.rangeDelIter.opSpanSeekGE("e") = e-f:{(#8,RANGEDEL)} -# 000000.rangeDelIter.opSpanSeekGE("e") = a-f:{(#8,RANGEDEL)} +# 000000.rangeDelIter.opSpanSeekGE("d") = a-e:{(#8,RANGEDEL)} +# 000001.rangeDelIter.opSpanSeekGE("e") = e-g:{(#8,RANGEDEL)} +# 000000.rangeDelIter.opSpanSeekGE("e") = nil # 000000.rangeDelIter.opSpanClose() = nil -# 000001.rangeDelIter.opSpanSeekGE("e") = e-f:{(#8,RANGEDEL)} +# 000001.rangeDelIter.opSpanSeekGE("e") = e-g:{(#8,RANGEDEL)} e#10,1:10 # 000001.rangeDelIter.opSpanSeekGE("g") = nil err=injected error @@ -91,13 +91,10 @@ e#10,1:10 c#27,1:27 a#30,1:30 -# We keep the rangedel alive by having a point in the higher level past the first point in the -# lower level. This rangedel hides that first point in the lower level but we should not seek to -# h and hide the second point. define L a.SET.15 f.SET.16 -a.SET.15:15 c.SET.13:13 f.SET.16:16 a.RANGEDEL.12:h +a.SET.15:15 c.SET.13:13 f.SET.16:16 a.RANGEDEL.12:f L e.SET.10 g.SET.15 e.SET.10:10 g.SET.15:15 @@ -136,18 +133,18 @@ f#16,1:16 c#13,1:13 a#15,1:15 -# The rangedel should not delete any points in any sstable. The two files were involved in an -# compaction and then the first file got moved to a lower level. +# The rangedel should not delete any points in any sstable. + define L -c.SET.30 f.RANGEDEL.0 -c.SET.30:30 d.SET.27:27 a.RANGEDEL.8:f +c.SET.30 f.RANGEDEL.72057594037927935 +c.SET.30:30 d.SET.27:27 e.RANGEDEL.8:f L a.SET.10 c.RANGEDEL.72057594037927935 -a.SET.10:10 b.SET.12:12 a.RANGEDEL.8:f +a.SET.10:10 b.SET.12:12 a.RANGEDEL.8:c ---- 1: - 000004:[c#30,SET-f#0,RANGEDEL] + 000004:[c#30,SET-f#inf,RANGEDEL] 2: 000005:[a#10,SET-c#inf,RANGEDEL] @@ -201,13 +198,10 @@ c#30,1:30 b#12,1:12 a#10,1:10 -# We keep the rangedel alive in prev iteration by having a point in the higher level before -# the last point in the lower level. This rangedel hides that first point in the lower level -# but we should not seek to a and hide the second point. define L c.SET.15 g.SET.16 -c.SET.15:15 f.SET.13:13 g.SET.16:16 a.RANGEDEL.12:h +c.SET.15:15 f.SET.13:13 g.SET.16:16 c.RANGEDEL.12:g L b.SET.14 d.SET.10 b.SET.14:14 d.SET.10:10 @@ -239,7 +233,7 @@ b#14,1:14 define L a.SET.30 e.RANGEDEL.72057594037927935 -a.SET.30:30 c.SET.27:27 a.RANGEDEL.8:g +a.SET.30:30 c.SET.27:27 a.RANGEDEL.8:e L e.SET.10 g.SET.20 e.SET.10:10 g.SET.20:20 e.RANGEDEL.8:g