Skip to content

Commit

Permalink
db: remove handling of untruncated range tombstones
Browse files Browse the repository at this point in the history
For a couple years now, Pebble has not allowed the creation of untruncated
range tombstones outside the context of virtual sstables. Additionally, since
v22.2 untruncated range tombstones should all have been compacted and rewritten
as truncated range tombstones. Virtual SSTables (which similarly allow a
physical range deletion to exceed the bounds of the containing logical sstable)
handle this case by truncating the tombstones at iteration time in the iterator
returned by keyspan.Truncate.

This leaves the merging iterator's delicate handling of untruncated range
tombstones obsolete. This commit removes that complexity. In addition, as an
extra safety precaution, the table stats collector's invariants-build assertion
that range deletions are contained within their files' bounds is lifted into
production builds as well. This provides a guarantee during store open that all
tombstones are appropriately truncated.

Relates to #2863.
  • Loading branch information
jbowens committed Jan 29, 2024
1 parent 5b280af commit eb8e9db
Show file tree
Hide file tree
Showing 5 changed files with 87 additions and 198 deletions.
19 changes: 6 additions & 13 deletions level_checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
}
Expand Down
5 changes: 0 additions & 5 deletions level_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
Expand Down
168 changes: 23 additions & 145 deletions merging_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand All @@ -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
Expand Down Expand Up @@ -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).
Expand All @@ -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
Expand Down Expand Up @@ -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
}
}
}
Expand Down Expand Up @@ -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
}
}
}
Expand Down
57 changes: 43 additions & 14 deletions table_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}

Expand Down
Loading

0 comments on commit eb8e9db

Please sign in to comment.