From 1253033ca533893323deb024a45aea80600197a2 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Wed, 16 Aug 2023 14:56:32 -0400 Subject: [PATCH] internal/keyspan: refactor interleaving iterator This commit refactors the interleaving iterator with a goal of simplifying its logic, but also to support a future mode in which the interleaving iterator may interleave end boundaries during iteration. The logic to interleave span bounds and point keys is disentangled from the logic to determine when to yield a key-value pair to the caller. This also allows the yielding logic to be shared among forward and reverse iteration. --- compaction.go | 6 +- compaction_iter_test.go | 2 +- db.go | 11 +- external_iterator.go | 8 +- internal/keyspan/interleaving_iter.go | 782 ++++++++---------- internal/keyspan/interleaving_iter_test.go | 13 +- internal/keyspan/testdata/interleaving_iter | 73 ++ .../testdata/interleaving_iter_masking | 1 - iterator.go | 3 +- range_keys.go | 6 +- scan_internal.go | 11 +- scan_internal_test.go | 2 +- 12 files changed, 467 insertions(+), 451 deletions(-) diff --git a/compaction.go b/compaction.go index 5959517c02..92180fe805 100644 --- a/compaction.go +++ b/compaction.go @@ -1244,7 +1244,7 @@ func (c *compaction) newInputIter( if rangeKeyIter := f.newRangeKeyIter(nil); rangeKeyIter != nil { mi := &keyspan.MergingIter{} mi.Init(c.cmp, rangeKeyCompactionTransform(c.equal, snapshots, c.elideRangeKey), new(keyspan.MergingBuffers), rangeKeyIter) - c.rangeKeyInterleaving.Init(c.comparer, iter, mi, nil /* hooks */, nil /* lowerBound */, nil /* upperBound */) + c.rangeKeyInterleaving.Init(c.comparer, iter, mi, keyspan.InterleavingIterOpts{}) iter = &c.rangeKeyInterleaving } return iter, nil @@ -1271,7 +1271,7 @@ func (c *compaction) newInputIter( if len(rangeKeyIters) > 0 { mi := &keyspan.MergingIter{} mi.Init(c.cmp, rangeKeyCompactionTransform(c.equal, snapshots, c.elideRangeKey), new(keyspan.MergingBuffers), rangeKeyIters...) - c.rangeKeyInterleaving.Init(c.comparer, iter, mi, nil /* hooks */, nil /* lowerBound */, nil /* upperBound */) + c.rangeKeyInterleaving.Init(c.comparer, iter, mi, keyspan.InterleavingIterOpts{}) iter = &c.rangeKeyInterleaving } return iter, nil @@ -1557,7 +1557,7 @@ func (c *compaction) newInputIter( mi.Init(c.cmp, rangeKeyCompactionTransform(c.equal, snapshots, c.elideRangeKey), new(keyspan.MergingBuffers), rangeKeyIters...) di := &keyspan.DefragmentingIter{} di.Init(c.comparer, mi, keyspan.DefragmentInternal, keyspan.StaticDefragmentReducer, new(keyspan.DefragmentingBuffers)) - c.rangeKeyInterleaving.Init(c.comparer, pointKeyIter, di, nil /* hooks */, nil /* lowerBound */, nil /* upperBound */) + c.rangeKeyInterleaving.Init(c.comparer, pointKeyIter, di, keyspan.InterleavingIterOpts{}) return &c.rangeKeyInterleaving, nil } diff --git a/compaction_iter_test.go b/compaction_iter_test.go index 131859d585..4282e52153 100644 --- a/compaction_iter_test.go +++ b/compaction_iter_test.go @@ -109,7 +109,7 @@ func TestCompactionIter(t *testing.T) { base.DefaultComparer, fi, keyspan.NewIter(base.DefaultComparer.Compare, rangeKeys), - nil, nil, nil) + keyspan.InterleavingIterOpts{}) iter := newInvalidatingIter(interleavingIter) iter.ignoreKind(InternalKeyKindRangeDelete) if merge == nil { diff --git a/db.go b/db.go index 8ad343aea7..4f60c05697 100644 --- a/db.go +++ b/db.go @@ -1168,7 +1168,11 @@ func finishInitializingIter(ctx context.Context, buf *iterAlloc) *Iterator { // dbi already had an initialized range key iterator, in case the point // iterator changed or the range key masking suffix changed. dbi.rangeKey.iiter.Init(&dbi.comparer, dbi.iter, dbi.rangeKey.rangeKeyIter, - &dbi.rangeKeyMasking, dbi.opts.LowerBound, dbi.opts.UpperBound) + keyspan.InterleavingIterOpts{ + Mask: &dbi.rangeKeyMasking, + LowerBound: dbi.opts.LowerBound, + UpperBound: dbi.opts.UpperBound, + }) dbi.iter = &dbi.rangeKey.iiter } } else { @@ -1314,7 +1318,10 @@ func finishInitializingInternalIter(buf *iterAlloc, i *scanInternalIterator) *sc // iterator that interleaves range keys pulled from // i.rangeKey.rangeKeyIter. i.rangeKey.iiter.Init(i.comparer, i.iter, i.rangeKey.rangeKeyIter, - nil /* mask */, i.opts.LowerBound, i.opts.UpperBound) + keyspan.InterleavingIterOpts{ + LowerBound: i.opts.LowerBound, + UpperBound: i.opts.UpperBound, + }) i.iter = &i.rangeKey.iiter return i diff --git a/external_iterator.go b/external_iterator.go index 8fbef76e72..7c70bd240c 100644 --- a/external_iterator.go +++ b/external_iterator.go @@ -319,8 +319,12 @@ func finishInitializingExternal(ctx context.Context, it *Iterator) { } } if it.rangeKey != nil { - it.rangeKey.iiter.Init(&it.comparer, it.iter, it.rangeKey.rangeKeyIter, &it.rangeKeyMasking, - it.opts.LowerBound, it.opts.UpperBound) + it.rangeKey.iiter.Init(&it.comparer, it.iter, it.rangeKey.rangeKeyIter, + keyspan.InterleavingIterOpts{ + Mask: &it.rangeKeyMasking, + LowerBound: it.opts.LowerBound, + UpperBound: it.opts.UpperBound, + }) it.iter = &it.rangeKey.iiter } } diff --git a/internal/keyspan/interleaving_iter.go b/internal/keyspan/interleaving_iter.go index a507b758be..8fce9e2709 100644 --- a/internal/keyspan/interleaving_iter.go +++ b/internal/keyspan/interleaving_iter.go @@ -111,6 +111,12 @@ type InterleavingIter struct { nextPrefixBuf []byte pointKey *base.InternalKey pointVal base.LazyValue + // prefix records the iterator's current prefix if the iterator is in prefix + // mode. During prefix mode, Pebble will truncate spans to the next prefix. + // If the iterator subsequently leaves prefix mode, the existing span cached + // in i.span must be invalidated because its bounds do not reflect the + // original span's true bounds. + prefix []byte // span holds the span at the keyspanIter's current position. If the span is // wholly contained within the iterator bounds, this span is directly // returned to the iterator consumer through Span(). If either bound needed @@ -126,31 +132,16 @@ type InterleavingIter struct { truncatedSpan Span truncated bool - // Keeping all of the bools together reduces the sizeof the struct. - - // spanCoversKey indicates whether the current span covers the last-returned - // key. - spanCoversKey bool - // pointKeyInterleaved indicates whether the current point key has been - // interleaved in the current direction. - pointKeyInterleaved bool - // keyspanInterleaved indicates whether or not the current span has been - // interleaved at its start key in the current direction. A span marker is - // interleaved when first passing over the start key. - // - // When iterating in the forward direction, the span start key is - // interleaved when the span first begins to cover the current iterator - // position. The keyspan iterator isn't advanced until the - // InterleavingIterator moves beyond the current span's end key. This field - // is used to remember that the span has already been interleaved and - // shouldn't be interleaved again. - // - // When iterating in the reverse direction, the span start key is - // interleaved immediately before the iterator will move to a key no longer - // be covered by the span. This field behaves analagously to - // pointKeyInterleaved and if true signals that we must Prev the keyspan - // iterator on the next Prev call. - keyspanInterleaved bool + // Keeping all of the bools/uint8s together reduces the sizeof the struct. + + // pos encodes the current position of the iterator: exhausted, on the point + // key, on a keyspan start, or on a keyspan end. + pos interleavePos + // withinSpan indicates whether the iterator is currently positioned within + // the bounds of the current span (i.span). withinSpan must be updated + // whenever the interleaving iterator's position enters or exits the bounds + // of a span. + withinSpan bool // spanMarkerTruncated is set by SeekGE/SeekPrefixGE calls that truncate a // span's start bound marker to the search key. It's returned to false on // the next repositioning of the keyspan iterator. @@ -158,39 +149,54 @@ type InterleavingIter struct { // maskSpanChangedCalled records whether or not the last call to // SpanMask.SpanChanged provided the current span (i.span) or not. maskSpanChangedCalled bool - // prefix records whether the iteator is in prefix mode. During prefix mode, - // Pebble will truncate spans to the next prefix. If the iterator - // subsequently leaves prefix mode, the existing span cached in i.span must - // be invalidated because its bounds do not reflect the original span's true - // bounds. - prefix bool // dir indicates the direction of iteration: forward (+1) or backward (-1) dir int8 } +// interleavePos indicates the iterator's current position. Note that both +// keyspanStart and keyspanEnd positions correspond to their user key boundaries +// with maximal sequence numbers. This means in the forward direction +// posKeyspanStart and posKeyspanEnd are always interleaved before a posPointKey +// with the same user key. +type interleavePos int8 + +const ( + posUninitialized interleavePos = iota + posExhausted + posPointKey + posKeyspanStart + posKeyspanEnd +) + // Assert that *InterleavingIter implements the InternalIterator interface. var _ base.InternalIterator = &InterleavingIter{} +// InterleavingIterOpts holds options configuring the behavior of a +// InterleavingIter. +type InterleavingIterOpts struct { + Mask SpanMask + LowerBound, UpperBound []byte +} + // Init initializes the InterleavingIter to interleave point keys from pointIter // with key spans from keyspanIter. // -// The point iterator must already have the provided bounds. Init does not -// propagate the bounds down the iterator stack. +// The point iterator must already have the bounds provided on opts. Init does +// not propagate the bounds down the iterator stack. func (i *InterleavingIter) Init( comparer *base.Comparer, pointIter base.InternalIterator, keyspanIter FragmentIterator, - mask SpanMask, - lowerBound, upperBound []byte, + opts InterleavingIterOpts, ) { *i = InterleavingIter{ cmp: comparer.Compare, comparer: comparer, pointIter: pointIter, keyspanIter: keyspanIter, - mask: mask, - lower: lowerBound, - upper: upperBound, + mask: opts.Mask, + lower: opts.LowerBound, + upper: opts.UpperBound, } } @@ -209,14 +215,14 @@ func (i *InterleavingIter) InitSeekGE( ) (*base.InternalKey, base.LazyValue) { i.dir = +1 i.clearMask() - i.prefix = prefix != nil + i.prefix = prefix i.pointKey, i.pointVal = pointKey, pointValue - i.pointKeyInterleaved = false // NB: This keyspanSeekGE call will truncate the span to the seek key if // necessary. This truncation is important for cases where a switch to // combined iteration is made during a user-initiated SeekGE. i.keyspanSeekGE(key, prefix) - return i.interleaveForward(key, prefix) + i.computeSmallestPos() + return i.yieldPosition(key, i.nextPos) } // InitSeekLT may be called after Init but before any positioning method. @@ -235,9 +241,9 @@ func (i *InterleavingIter) InitSeekLT( i.dir = -1 i.clearMask() i.pointKey, i.pointVal = pointKey, pointValue - i.pointKeyInterleaved = false i.keyspanSeekLT(key) - return i.interleaveBackward() + i.computeLargestPos() + return i.yieldPosition(i.lower, i.prevPos) } // SeekGE implements (base.InternalIterator).SeekGE. @@ -257,7 +263,6 @@ func (i *InterleavingIter) SeekGE( i.clearMask() i.disablePrefixMode() i.pointKey, i.pointVal = i.pointIter.SeekGE(key, flags) - i.pointKeyInterleaved = false // We need to seek the keyspan iterator too. If the keyspan iterator was // already positioned at a span, we might be able to avoid the seek if the @@ -265,14 +270,15 @@ func (i *InterleavingIter) SeekGE( if i.span != nil && i.cmp(key, i.span.End) < 0 && i.cmp(key, i.span.Start) >= 0 { // We're seeking within the existing span's bounds. We still might need // truncate the span to the iterator's bounds. - i.checkForwardBound(nil /* prefix */) + i.checkForwardBound() i.savedKeyspan() } else { i.keyspanSeekGE(key, nil /* prefix */) } i.dir = +1 - return i.interleaveForward(key, nil /* prefix */) + i.computeSmallestPos() + return i.yieldPosition(key, i.nextPos) } // SeekPrefixGE implements (base.InternalIterator).SeekPrefixGE. @@ -290,9 +296,8 @@ func (i *InterleavingIter) SeekPrefixGE( prefix, key []byte, flags base.SeekGEFlags, ) (*base.InternalKey, base.LazyValue) { i.clearMask() + i.prefix = prefix i.pointKey, i.pointVal = i.pointIter.SeekPrefixGE(prefix, key, flags) - i.pointKeyInterleaved = false - i.prefix = true // We need to seek the keyspan iterator too. If the keyspan iterator was // already positioned at a span, we might be able to avoid the seek if the @@ -321,7 +326,7 @@ func (i *InterleavingIter) SeekPrefixGE( if ei := i.comparer.Split(i.span.End); i.cmp(prefix, i.span.End[:ei]) < 0 { // We're seeking within the existing span's bounds. We still might need // truncate the span to the iterator's bounds. - i.checkForwardBound(prefix) + i.checkForwardBound() i.savedKeyspan() seekKeyspanIter = false } @@ -331,7 +336,8 @@ func (i *InterleavingIter) SeekPrefixGE( } i.dir = +1 - return i.interleaveForward(key, prefix) + i.computeSmallestPos() + return i.yieldPosition(key, i.nextPos) } // SeekLT implements (base.InternalIterator).SeekLT. @@ -341,7 +347,6 @@ func (i *InterleavingIter) SeekLT( i.clearMask() i.disablePrefixMode() i.pointKey, i.pointVal = i.pointIter.SeekLT(key, flags) - i.pointKeyInterleaved = false // We need to seek the keyspan iterator too. If the keyspan iterator was // already positioned at a span, we might be able to avoid the seek if the @@ -371,7 +376,8 @@ func (i *InterleavingIter) SeekLT( } i.dir = -1 - return i.interleaveBackward() + i.computeLargestPos() + return i.yieldPosition(i.lower, i.prevPos) } // First implements (base.InternalIterator).First. @@ -379,12 +385,12 @@ func (i *InterleavingIter) First() (*base.InternalKey, base.LazyValue) { i.clearMask() i.disablePrefixMode() i.pointKey, i.pointVal = i.pointIter.First() - i.pointKeyInterleaved = false i.span = i.keyspanIter.First() - i.checkForwardBound(nil /* prefix */) + i.checkForwardBound() i.savedKeyspan() i.dir = +1 - return i.interleaveForward(i.lower, nil /* prefix */) + i.computeSmallestPos() + return i.yieldPosition(i.lower, i.nextPos) } // Last implements (base.InternalIterator).Last. @@ -392,12 +398,12 @@ func (i *InterleavingIter) Last() (*base.InternalKey, base.LazyValue) { i.clearMask() i.disablePrefixMode() i.pointKey, i.pointVal = i.pointIter.Last() - i.pointKeyInterleaved = false i.span = i.keyspanIter.Last() i.checkBackwardBound() i.savedKeyspan() i.dir = -1 - return i.interleaveBackward() + i.computeLargestPos() + return i.yieldPosition(i.lower, i.prevPos) } // Next implements (base.InternalIterator).Next. @@ -413,70 +419,43 @@ func (i *InterleavingIter) Next() (*base.InternalKey, base.LazyValue) { i.clearMask() } - // The existing point key (denoted below with *) is either the last - // key we returned (the current iterator position): - // points: x (y*) z - // or the upcoming point key in the backward direction if we just - // returned a span start boundary key: - // points: x* z - // spans: ([y-?)) - // direction. Either way, we must move to the next point key. - switch { - case i.pointKey == nil && i.lower == nil: - i.pointKey, i.pointVal = i.pointIter.First() - case i.pointKey == nil && i.lower != nil: - i.pointKey, i.pointVal = i.pointIter.SeekGE(i.lower, base.SeekGEFlagsNone) - default: + // When switching directions, iterator state corresponding to the + // current iterator position (as indicated by i.pos) is already correct. + // However any state that has yet to be interleaved describes a position + // behind the current iterator position and needs to be updated to + // describe the position ahead of the current iterator position. + switch i.pos { + case posExhausted: + // Nothing to do. The below nextPos call will move both the point + // key and span to their next positions and return + // MIN(point,s.Start). + case posPointKey: + // If we're currently on a point key, the below nextPos will + // correctly Next the point key iterator to the next point key. + // Do we need to move the span forwards? If the current span lies + // entirely behind the current key (!i.withinSpan), then we + // need to move it to the first span in the forward direction. + if !i.withinSpan { + i.span = i.keyspanIter.Next() + i.checkForwardBound() + i.savedKeyspan() + } + case posKeyspanStart: + i.withinSpan = true + // Since we're positioned on a Span, the pointIter is positioned + // entirely behind the current iterator position. Reposition it + // ahead of the current iterator position. + i.pointKey, i.pointVal = i.pointIter.Next() + case posKeyspanEnd: + // Since we're positioned on a Span, the pointIter is positioned + // entirely behind of the current iterator position. Reposition it + // ahead the current iterator position. i.pointKey, i.pointVal = i.pointIter.Next() } - i.pointKeyInterleaved = false - - if i.span == nil { - // There was no span in the reverse direction, but there may be - // a span in the forward direction. - i.span = i.keyspanIter.Next() - i.checkForwardBound(nil /* prefix */) - i.savedKeyspan() - } else { - // Regardless of the current iterator state, we mark any existing - // span as interleaved when switching to forward iteration, - // justified below. - // - // If the point key is the last key returned: - // pointIter : ... (y) z ... - // keyspanIter : ... ([x - )) ... - // ^ - // The span's start key must be ≤ the point key, otherwise we'd have - // interleaved the span's start key. From a forward-iteration - // perspective, the span's start key is in the past and should be - // considered already-interleaved. - // - // If the span start boundary key is the last key returned: - // pointIter : ... (x) z ... - // keyspanIter : ... ([y - )) ... - // ^ - // i.span.Start is the key we last returned during reverse - // iteration. From the perspective of forward-iteration, its start - // key was just visited. - i.keyspanInterleaved = true - } + // Fallthrough to calling i.nextPos. } - - // Refresh the point key if the current point key has already been - // interleaved. - if i.pointKeyInterleaved { - i.pointKey, i.pointVal = i.pointIter.Next() - i.pointKeyInterleaved = false - } - // If we already interleaved the current span start key, and the point key - // is ≥ the span's end key, move to the next span. - if i.keyspanInterleaved && i.pointKey != nil && i.span != nil && - i.cmp(i.pointKey.UserKey, i.span.End) >= 0 { - i.span = i.keyspanIter.Next() - i.checkForwardBound(nil /* prefix */) - i.savedKeyspan() - } - return i.interleaveForward(i.lower, nil /* prefix */) + i.nextPos() + return i.yieldPosition(i.lower, i.nextPos) } // NextPrefix implements (base.InternalIterator).NextPrefix. @@ -485,22 +464,24 @@ func (i *InterleavingIter) NextPrefix(succKey []byte) (*base.InternalKey, base.L panic("pebble: cannot switch directions with NextPrefix") } - // Refresh the point key if the current point key has already been - // interleaved, OR if the next uninterleaved point key has the old suffix - // (it's less than succKey). - if i.pointKeyInterleaved || (i.pointKey != nil && i.cmp(i.pointKey.UserKey, succKey) < 0) { + switch i.pos { + case posExhausted: + return nil, base.LazyValue{} + case posPointKey: i.pointKey, i.pointVal = i.pointIter.NextPrefix(succKey) - i.pointKeyInterleaved = false - } - // If we already interleaved the current span start key, and the point key - // is ≥ the span's end key, move to the next span. - if i.keyspanInterleaved && i.pointKey != nil && i.span != nil && - i.cmp(i.pointKey.UserKey, i.span.End) >= 0 { - i.span = i.keyspanIter.Next() - i.checkForwardBound(nil) - i.savedKeyspan() + if i.withinSpan { + if i.pointKey == nil || i.cmp(i.span.End, i.pointKey.UserKey) <= 0 { + i.pos = posKeyspanEnd + } else { + i.pos = posPointKey + } + } else { + i.computeSmallestPos() + } + case posKeyspanStart, posKeyspanEnd: + i.nextPos() } - return i.interleaveForward(i.lower, nil) + return i.yieldPosition(i.lower, i.nextPos) } // Prev implements (base.InternalIterator).Prev. @@ -516,18 +497,35 @@ func (i *InterleavingIter) Prev() (*base.InternalKey, base.LazyValue) { i.clearMask() } - if i.keyspanInterleaved { - // The current span's start key has already been interleaved in the - // forward direction. The start key may have been interleaved a - // while ago, or it might've been interleaved at the current - // iterator position. If it was interleaved a while ago, the current - // span is still relevant and we should not move the keyspan - // iterator. - // - // If it was just interleaved at the current iterator position, the - // span start was the last key returned to the user. We should - // prev past it so we don't return it again, with an exception. - // Consider span [a, z) and this sequence of iterator calls: + // When switching directions, iterator state corresponding to the + // current iterator position (as indicated by i.pos) is already correct. + // However any state that has yet to be interleaved describes a position + // ahead of the current iterator position and needs to be updated to + // describe the position behind the current iterator position. + switch i.pos { + case posExhausted: + // Nothing to do. The below prevPos call will move both the point + // key and span to previous positions and return MAX(point, s.End). + case posPointKey: + // If we're currently on a point key, the point iterator is in the + // right place and the call to prevPos will correctly Prev the point + // key iterator to the previous point key. Do we need to move the + // span backwards? If the current span lies entirely ahead of the + // current key (!i.withinSpan), then we need to move it to the first + // span in the reverse direction. + if !i.withinSpan { + i.span = i.keyspanIter.Prev() + i.checkBackwardBound() + i.savedKeyspan() + } + case posKeyspanStart: + // Since we're positioned on a Span, the pointIter is positioned + // entirely ahead of the current iterator position. Reposition it + // behind the current iterator position. + i.pointKey, i.pointVal = i.pointIter.Prev() + // Without considering truncation of spans to seek keys, the keyspan + // iterator is already in the right place. But consider span [a, z) + // and this sequence of iterator calls: // // SeekGE('c') = c.RANGEKEYSET#72057594037927935 // Prev() = a.RANGEKEYSET#72057594037927935 @@ -535,176 +533,190 @@ func (i *InterleavingIter) Prev() (*base.InternalKey, base.LazyValue) { // If the current span's start key was last surfaced truncated due // to a SeekGE or SeekPrefixGE call, then it's still relevant in the // reverse direction with an untruncated start key. - // - // We can determine whether the last key returned was a point key by - // checking i.pointKeyInterleaved, because every Next/Prev will - // advance the point iterator and reset pointKeyInterleaved if it - // was. - if i.pointKeyInterleaved || i.spanMarkerTruncated { - // The last returned key was a point key, OR a truncated span - // marker key. Don't move, but re-save the span because it - // should no longer be considered truncated or interleaved. - i.savedKeyspan() - } else { - // The last returned key is this key's start boundary, so Prev - // past it so we don't return it again. - i.span = i.keyspanIter.Prev() - i.checkBackwardBound() - i.savedKeyspan() + if i.spanMarkerTruncated { + // When we fallthrough to calling prevPos, we want to move to + // MAX(point, span.Start). We cheat here by claiming we're + // currently on the end boundary, so that we'll move on to the + // untruncated start key if necessary. + i.pos = posKeyspanEnd } - } else { - // If the current span's start key has not been interleaved, then - // the span's start key is greater than the current iterator - // position (denoted in parenthesis), and the current span's start - // key is ahead of our iterator position. Move it to the previous - // span: - // points: (x*) - // span: [y-z)* - i.span = i.keyspanIter.Prev() - i.checkBackwardBound() + case posKeyspanEnd: + // Since we're positioned on a Span, the pointIter is positioned + // entirely ahead of the current iterator position. Reposition it + // behind the current iterator position. + i.pointKey, i.pointVal = i.pointIter.Prev() + } + + if i.spanMarkerTruncated { + // Save the keyspan again to clear truncation. i.savedKeyspan() } + // Fallthrough to calling i.prevPos. + } + i.prevPos() + return i.yieldPosition(i.lower, i.prevPos) +} - // The existing point key (denoted below with *) is either the last - // key we returned (the current iterator position): - // points: x (y*) z - // or the upcoming point key in the forward direction if we just - // returned a span start boundary key : - // points: x z* - // spans: ([y-?)) - // direction. Either way, we must move the point iterator backwards. +// computeSmallestPos sets i.{pos,withinSpan} to: +// +// MIN(i.pointKey, i.span.Start) +func (i *InterleavingIter) computeSmallestPos() { + if i.span != nil && (i.pointKey == nil || i.cmp(i.startKey(), i.pointKey.UserKey) <= 0) { + i.withinSpan = true + i.pos = posKeyspanStart + return + } + i.withinSpan = false + if i.pointKey != nil { + i.pos = posPointKey + return + } + i.pos = posExhausted +} + +// computeLargestPos sets i.{pos,withinSpan} to: +// +// MAX(i.pointKey, i.span.End) +func (i *InterleavingIter) computeLargestPos() { + if i.span != nil && (i.pointKey == nil || i.cmp(i.span.End, i.pointKey.UserKey) > 0) { + i.withinSpan = true + i.pos = posKeyspanEnd + return + } + i.withinSpan = false + if i.pointKey != nil { + i.pos = posPointKey + return + } + i.pos = posExhausted +} + +// nextPos advances the iterator one position in the forward direction. +func (i *InterleavingIter) nextPos() { + switch i.pos { + case posExhausted: + i.pointKey, i.pointVal = i.pointIter.Next() + i.span = i.keyspanIter.Next() + i.checkForwardBound() + i.savedKeyspan() + i.computeSmallestPos() + case posPointKey: + i.pointKey, i.pointVal = i.pointIter.Next() + // If we're not currently within the span, we want to chose the + // MIN(pointKey,span.Start), which is exactly the calculation performed + // by computeSmallestPos. + if !i.withinSpan { + i.computeSmallestPos() + return + } + // i.withinSpan=true + // Since we previously were within the span, we want to choose the + // MIN(pointKey,span.End). switch { - case i.pointKey == nil && i.upper == nil: - i.pointKey, i.pointVal = i.pointIter.Last() - case i.pointKey == nil && i.upper != nil: - i.pointKey, i.pointVal = i.pointIter.SeekLT(i.upper, base.SeekLTFlagsNone) + case i.span == nil: + panic("i.withinSpan=true and i.span=nil") + case i.pointKey == nil: + // Since i.withinSpan=true, we step onto the end boundary of the + // keyspan. + i.pos = posKeyspanEnd default: - i.pointKey, i.pointVal = i.pointIter.Prev() + // i.withinSpan && i.pointKey != nil && i.span != nil + if i.cmp(i.span.End, i.pointKey.UserKey) <= 0 { + i.pos = posKeyspanEnd + } else { + i.pos = posPointKey + } } - i.pointKeyInterleaved = false + case posKeyspanStart: + // Either a point key or the span's end key comes next. + if i.pointKey != nil && i.cmp(i.pointKey.UserKey, i.span.End) < 0 { + i.pos = posPointKey + } else { + i.pos = posKeyspanEnd + } + case posKeyspanEnd: + i.span = i.keyspanIter.Next() + i.checkForwardBound() + i.savedKeyspan() + i.computeSmallestPos() + default: + panic(fmt.Sprintf("unexpected pos=%d\n", i.pos)) } +} - // Refresh the point key if we just returned the current point key. - if i.pointKeyInterleaved { +// prevPos advances the iterator one position in the reverse direction. +func (i *InterleavingIter) prevPos() { + switch i.pos { + case posExhausted: i.pointKey, i.pointVal = i.pointIter.Prev() - i.pointKeyInterleaved = false - } - // Refresh the span if we just returned the span's start boundary key. - if i.keyspanInterleaved { i.span = i.keyspanIter.Prev() i.checkBackwardBound() i.savedKeyspan() + i.computeLargestPos() + case posPointKey: + i.pointKey, i.pointVal = i.pointIter.Prev() + // If we're not currently covered by the span, we want to chose the + // MAX(pointKey,span.End), which is exactly the calculation performed + // by computeLargestPos. + if !i.withinSpan { + i.computeLargestPos() + return + } + switch { + case i.span == nil: + panic("withinSpan=true, but i.span == nil") + case i.pointKey == nil: + i.pos = posKeyspanEnd + default: + // i.withinSpan && i.pointKey != nil && i.span != nil + if i.cmp(i.span.Start, i.pointKey.UserKey) > 0 { + i.pos = posKeyspanStart + } else { + i.pos = posPointKey + } + } + case posKeyspanStart: + i.span = i.keyspanIter.Prev() + i.checkBackwardBound() + i.savedKeyspan() + i.computeLargestPos() + case posKeyspanEnd: + // Either a point key or the span's start key is previous. + if i.pointKey != nil && i.cmp(i.pointKey.UserKey, i.span.Start) >= 0 { + i.pos = posPointKey + } else { + i.pos = posKeyspanStart + } + default: + panic(fmt.Sprintf("unexpected pos=%d\n", i.pos)) } - return i.interleaveBackward() } -func (i *InterleavingIter) interleaveForward( - lowerBound []byte, prefix []byte, +func (i *InterleavingIter) yieldPosition( + lowerBound []byte, advance func(), ) (*base.InternalKey, base.LazyValue) { - // This loop determines whether a point key or a span marker key should be - // interleaved on each iteration. If masking is disabled and the span is - // nonempty, this loop executes for exactly one iteration. If masking is - // enabled and a masked key is determined to be interleaved next, this loop - // continues until the interleaved key is unmasked. If a span's start key - // should be interleaved next, but the span is empty, the loop continues to - // the next key. + // This loop returns the first visible position in the current iteration + // direction. Some positions are not visible and skipped. For example, if + // masking is enabled and the iterator is positioned over a masked point + // key, this loop skips the position. If a span's start key should be + // interleaved next, but the span is empty, the loop continues to the next + // key. Currently, span end keys are also always skipped, and are used only + // for maintaining internal state. for { - // Check invariants. - if invariants.Enabled { - // INVARIANT: !pointKeyInterleaved - if i.pointKeyInterleaved { - panic("pebble: invariant violation: point key interleaved") - } - switch { - case i.span == nil: - case i.pointKey == nil: - default: - // INVARIANT: !keyspanInterleaved || pointKey < span.End - // The caller is responsible for advancing this span if it's already - // been interleaved and the span ends before the point key. - // Absolute positioning methods will never have already interleaved - // the span's start key, so only Next needs to handle the case where - // pointKey >= span.End. - if i.keyspanInterleaved && i.cmp(i.pointKey.UserKey, i.span.End) >= 0 { - panic("pebble: invariant violation: span interleaved, but point key >= span end") - } + switch i.pos { + case posExhausted: + return i.yieldNil() + case posPointKey: + if i.pointKey == nil { + panic("i.pointKey is nil") } - } - // Interleave. - switch { - case i.span == nil: - // If we're out of spans, just return the point key. - return i.yieldPointKey(false /* covered */) - case i.pointKey == nil: - if i.pointKeyInterleaved { - panic("pebble: invariant violation: point key already interleaved") - } - // If we're out of point keys, we need to return a span marker. If - // the current span has already been interleaved, advance it. Since - // there are no more point keys, we don't need to worry about - // advancing past the current point key. - if i.keyspanInterleaved { - i.span = i.keyspanIter.Next() - i.checkForwardBound(prefix) - i.savedKeyspan() - if i.span == nil { - return i.yieldNil() - } - } - if i.span.Empty() { - i.keyspanInterleaved = true - continue - } - return i.yieldSyntheticSpanMarker(lowerBound) - default: - if i.cmp(i.pointKey.UserKey, i.startKey()) >= 0 { - // The span start key lies before the point key. If we haven't - // interleaved it, we should. - if !i.keyspanInterleaved { - if i.span.Empty() { - if i.pointKey != nil && i.cmp(i.pointKey.UserKey, i.span.End) >= 0 { - // Advance the keyspan iterator, as just flipping - // keyspanInterleaved would likely trip up the invariant check - // above. - i.span = i.keyspanIter.Next() - i.checkForwardBound(prefix) - i.savedKeyspan() - } else { - i.keyspanInterleaved = true - } - continue - } - return i.yieldSyntheticSpanMarker(lowerBound) - } - - // Otherwise, the span's start key is already interleaved and we - // need to return the point key. The current span necessarily - // must cover the point key: - // - // Since the span's start is less than or equal to the point - // key, the only way for this span to not cover the point would - // be if the span's end is less than or equal to the point. - // (For example span = [a, b), point key = c). - // - // However, the invariant at the beginning of the function - // guarantees that if: - // * we have both a point key and a span - // * and the span has already been interleaved - // => then the point key must be less than the span's end, and - // the point key must be covered by the current span. - - // The span covers the point key. If a SkipPoint hook is - // configured, ask it if we should skip this point key. - // - // But first, we may need to update the mask to the current span - // if we have stepped outside of the span last saved as a mask, - // so that the decision to skip is made with the correct - // knowledge of the covering span. - i.maybeUpdateMask(true /* covered */) - - if i.mask != nil && i.mask.SkipPoint(i.pointKey.UserKey) { - if i.prefix { + if i.mask != nil { + i.maybeUpdateMask() + if i.withinSpan && i.mask.SkipPoint(i.pointKey.UserKey) { + // The span covers the point key. If a SkipPoint hook is + // configured, ask it if we should skip this point key. + if i.prefix != nil { // During prefix-iteration node, once a point is masked, // all subsequent keys with the same prefix must also be // masked according to the key ordering. We can stop and @@ -721,105 +733,27 @@ func (i *InterleavingIter) interleaveForward( // TODO(jackson): If we thread a base.Comparer through to // InterleavingIter so that we have access to // ImmediateSuccessor, we could use NextPrefix. We'd need to - // tweak the SpanMask interface slightly, but it's probably - // worthwhile. - - i.pointKey, i.pointVal = i.pointIter.Next() - // We may have just invalidated the invariant that - // ensures the span's End is > the point key, so - // reestablish it before the next iteration. - if i.pointKey != nil && i.cmp(i.pointKey.UserKey, i.span.End) >= 0 { - i.span = i.keyspanIter.Next() - i.checkForwardBound(prefix) - i.savedKeyspan() - } + // tweak the SpanMask interface slightly. + + // Advance beyond the masked point key. + advance() continue } - - // Point key is unmasked but covered. - return i.yieldPointKey(true /* covered */) - } - return i.yieldPointKey(false /* covered */) - } - } -} - -func (i *InterleavingIter) interleaveBackward() (*base.InternalKey, base.LazyValue) { - // This loop determines whether a point key or a span's start key should be - // interleaved on each iteration. If masking is disabled and the span is - // nonempty, this loop executes for exactly one iteration. If masking is - // enabled and a masked key is determined to be interleaved next, this loop - // continues until the interleaved key is unmasked. If a span's start key - // should be interleaved next, but the span is empty, the loop continues to - // the next key. - for { - // Check invariants. - if invariants.Enabled { - // INVARIANT: !pointKeyInterleaved - if i.pointKeyInterleaved { - panic("pebble: invariant violation: point key interleaved") } - } - - // Interleave. - switch { - case i.span == nil: - // If we're out of spans, just return the point key. - return i.yieldPointKey(false /* covered */) - case i.pointKey == nil: - // If we're out of point keys, we need to return a span marker. + return i.yieldPointKey() + case posKeyspanEnd: + // Don't interleave end keys; just advance. + advance() + continue + case posKeyspanStart: + // Don't interleave an empty span. if i.span.Empty() { - i.span = i.keyspanIter.Prev() - i.checkBackwardBound() - i.savedKeyspan() + advance() continue } - return i.yieldSyntheticSpanMarker(i.lower) + return i.yieldSyntheticSpanMarker(lowerBound) default: - // If the span's start key is greater than the point key, return a - // marker for the span. - if i.cmp(i.startKey(), i.pointKey.UserKey) > 0 { - if i.span.Empty() { - i.span = i.keyspanIter.Prev() - i.checkBackwardBound() - i.savedKeyspan() - continue - } - return i.yieldSyntheticSpanMarker(i.lower) - } - // We have a span but it has not been interleaved and begins at a - // key equal to or before the current point key. The point key - // should be interleaved next, if it's not masked. - if i.cmp(i.pointKey.UserKey, i.span.End) < 0 { - // The span covers the point key. The point key might be masked - // too if masking is enabled. - // - // The span may have changed since the last time we updated the - // mask. Consider the following range-key masking scenario: - // - // |--------------) [b,d)@5 - // . c@4 . e@9 - // - // During reverse iteration when we step from e@9 to c@4, we - // enter the span [b,d)@5. Since end boundaries are not - // interleaved, the mask hasn't been updated with the span - // [b,d)@5 yet. We must update the mask before calling - // SkipPoint(c@4) to maintain the SpanMask contract and give the - // mask implementation an opportunity to build the state - // necessary to be able to determine whether [b,d)@5 masks c@4. - i.maybeUpdateMask(true /* covered */) - - // The span covers the point key. If a SkipPoint hook is - // configured, ask it if we should skip this point key. - if i.mask != nil && i.mask.SkipPoint(i.pointKey.UserKey) { - i.pointKey, i.pointVal = i.pointIter.Prev() - continue - } - - // Point key is unmasked but covered. - return i.yieldPointKey(true /* covered */) - } - return i.yieldPointKey(false /* covered */) + panic(fmt.Sprintf("unexpected interleavePos=%d", i.pos)) } } } @@ -827,7 +761,7 @@ func (i *InterleavingIter) interleaveBackward() (*base.InternalKey, base.LazyVal // keyspanSeekGE seeks the keyspan iterator to the first span covering a key ≥ k. func (i *InterleavingIter) keyspanSeekGE(k []byte, prefix []byte) { i.span = i.keyspanIter.SeekGE(k) - i.checkForwardBound(prefix) + i.checkForwardBound() i.savedKeyspan() } @@ -852,7 +786,7 @@ func (i *InterleavingIter) keyspanSeekLT(k []byte) { i.savedKeyspan() } -func (i *InterleavingIter) checkForwardBound(prefix []byte) { +func (i *InterleavingIter) checkForwardBound() { i.truncated = false i.truncatedSpan = Span{} if i.span == nil { @@ -886,15 +820,15 @@ func (i *InterleavingIter) checkForwardBound(prefix []byte) { } // If this is a part of a SeekPrefixGE call, we may also need to truncate to // the prefix's bounds. - if prefix != nil { + if i.prefix != nil { if !i.truncated { i.truncated = true i.truncatedSpan = *i.span } - if i.cmp(prefix, i.truncatedSpan.Start) > 0 { - i.truncatedSpan.Start = prefix + if i.cmp(i.prefix, i.truncatedSpan.Start) > 0 { + i.truncatedSpan.Start = i.prefix } - i.nextPrefixBuf = i.comparer.ImmediateSuccessor(i.nextPrefixBuf[:0], prefix) + i.nextPrefixBuf = i.comparer.ImmediateSuccessor(i.nextPrefixBuf[:0], i.prefix) if i.truncated && i.cmp(i.nextPrefixBuf, i.truncatedSpan.End) < 0 { i.truncatedSpan.End = i.nextPrefixBuf } @@ -943,15 +877,12 @@ func (i *InterleavingIter) checkBackwardBound() { } func (i *InterleavingIter) yieldNil() (*base.InternalKey, base.LazyValue) { - i.spanCoversKey = false + i.withinSpan = false i.clearMask() return i.verify(nil, base.LazyValue{}) } -func (i *InterleavingIter) yieldPointKey(covered bool) (*base.InternalKey, base.LazyValue) { - i.pointKeyInterleaved = true - i.spanCoversKey = covered - i.maybeUpdateMask(covered) +func (i *InterleavingIter) yieldPointKey() (*base.InternalKey, base.LazyValue) { return i.verify(i.pointKey, i.pointVal) } @@ -960,8 +891,6 @@ func (i *InterleavingIter) yieldSyntheticSpanMarker( ) (*base.InternalKey, base.LazyValue) { i.spanMarker.UserKey = i.startKey() i.spanMarker.Trailer = base.MakeTrailer(base.InternalKeySeqNumMax, i.span.Keys[0].Kind()) - i.keyspanInterleaved = true - i.spanCoversKey = true // Truncate the key we return to our lower bound if we have one. Note that // we use the lowerBound function parameter, not i.lower. The lowerBound @@ -992,13 +921,13 @@ func (i *InterleavingIter) yieldSyntheticSpanMarker( i.spanMarker.UserKey = i.keyBuf i.spanMarkerTruncated = true } - i.maybeUpdateMask(true /* covered */) + i.maybeUpdateMask() return i.verify(&i.spanMarker, base.LazyValue{}) } func (i *InterleavingIter) disablePrefixMode() { - if i.prefix { - i.prefix = false + if i.prefix != nil { + i.prefix = nil // Clear the existing span. It may not hold the true end bound of the // underlying span. i.span = nil @@ -1012,52 +941,36 @@ func (i *InterleavingIter) verify( // production builds elide this entire function. if invariants.Enabled { switch { - case k != nil && !i.keyspanInterleaved && !i.pointKeyInterleaved: - panic("pebble: invariant violation: both keys marked as noninterleaved") - case i.dir == -1 && k != nil && i.keyspanInterleaved == i.pointKeyInterleaved: - // During reverse iteration, if we're returning a key, either the span's - // start key must have been interleaved OR the current point key value - // is being returned, not both. - // - // This invariant holds because in reverse iteration the start key of the - // span behaves like a point. Once the start key is interleaved, we move - // the keyspan iterator to the previous span. - panic(fmt.Sprintf("pebble: invariant violation: interleaving (point %t, span %t)", - i.pointKeyInterleaved, i.keyspanInterleaved)) case i.dir == -1 && i.spanMarkerTruncated: panic("pebble: invariant violation: truncated span key in reverse iteration") case k != nil && i.lower != nil && i.cmp(k.UserKey, i.lower) < 0: panic("pebble: invariant violation: key < lower bound") case k != nil && i.upper != nil && i.cmp(k.UserKey, i.upper) >= 0: panic("pebble: invariant violation: key ≥ upper bound") - case i.span != nil && k != nil && i.mask != nil && i.pointKeyInterleaved && - i.cmp(k.UserKey, i.span.Start) >= 0 && i.cmp(k.UserKey, i.span.End) < 0 && i.mask.SkipPoint(k.UserKey): - panic("pebble: invariant violation: point key eligible for skipping returned") } } return k, v } func (i *InterleavingIter) savedKeyspan() { - i.keyspanInterleaved = false i.spanMarkerTruncated = false i.maskSpanChangedCalled = false } -// maybeUpdateMask updates the current mask, if a mask is configured and -// the mask hasn't been updated with the current keyspan yet. -func (i *InterleavingIter) maybeUpdateMask(covered bool) { - if i.mask != nil { - if !covered || i.span.Empty() { - i.clearMask() - } else if !i.maskSpanChangedCalled { - if i.truncated { - i.mask.SpanChanged(&i.truncatedSpan) - } else { - i.mask.SpanChanged(i.span) - } - i.maskSpanChangedCalled = true - } +// updateMask updates the current mask, if a mask is configured and the mask +// hasn't been updated with the current keyspan yet. +func (i *InterleavingIter) maybeUpdateMask() { + switch { + case i.mask == nil, i.maskSpanChangedCalled: + return + case !i.withinSpan || i.span.Empty(): + i.clearMask() + case i.truncated: + i.mask.SpanChanged(&i.truncatedSpan) + i.maskSpanChangedCalled = true + default: + i.mask.SpanChanged(i.span) + i.maskSpanChangedCalled = true } } @@ -1084,13 +997,12 @@ func (i *InterleavingIter) startKey() []byte { // // Span will never return an invalid or empty span. func (i *InterleavingIter) Span() *Span { - if !i.spanCoversKey || i.span.Empty() { + if !i.withinSpan || len(i.span.Keys) == 0 { return nil } else if i.truncated { return &i.truncatedSpan - } else { - return i.span } + return i.span } // SetBounds implements (base.InternalIterator).SetBounds. diff --git a/internal/keyspan/interleaving_iter_test.go b/internal/keyspan/interleaving_iter_test.go index 41eb51d625..564857b30d 100644 --- a/internal/keyspan/interleaving_iter_test.go +++ b/internal/keyspan/interleaving_iter_test.go @@ -79,11 +79,13 @@ func runInterleavingIterTest(t *testing.T, filename string) { split: testkeys.Comparer.Split, } + var prevKey *base.InternalKey formatKey := func(k *base.InternalKey, _ base.LazyValue) { if k == nil { fmt.Fprint(&buf, ".") return } + prevKey = k s := iter.Span() fmt.Fprintf(&buf, "PointKey: %s\n", k.String()) if s != nil { @@ -107,7 +109,8 @@ func runInterleavingIterTest(t *testing.T, filename string) { } keyspanIter.Init(cmp, noopTransform, new(MergingBuffers), NewIter(cmp, spans)) hooks.maskSuffix = nil - iter.Init(testkeys.Comparer, &pointIter, &keyspanIter, &hooks, nil, nil) + iter.Init(testkeys.Comparer, &pointIter, &keyspanIter, + InterleavingIterOpts{Mask: &hooks}) return "OK" case "define-pointkeys": var points []base.InternalKey @@ -117,12 +120,14 @@ func runInterleavingIterTest(t *testing.T, filename string) { } pointIter = pointIterator{cmp: cmp, keys: points} hooks.maskSuffix = nil - iter.Init(testkeys.Comparer, &pointIter, &keyspanIter, &hooks, nil, nil) + iter.Init(testkeys.Comparer, &pointIter, &keyspanIter, + InterleavingIterOpts{Mask: &hooks}) return "OK" case "iter": buf.Reset() // Clear any previous bounds. iter.SetBounds(nil, nil) + prevKey = nil lines := strings.Split(strings.TrimSpace(td.Input), "\n") for _, line := range lines { bufLen := buf.Len() @@ -139,6 +144,9 @@ func runInterleavingIterTest(t *testing.T, filename string) { formatKey(iter.Last()) case "next": formatKey(iter.Next()) + case "next-prefix": + succKey := testkeys.Comparer.ImmediateSuccessor(nil, prevKey.UserKey[:testkeys.Comparer.Split(prevKey.UserKey)]) + formatKey(iter.NextPrefix(succKey)) case "prev": formatKey(iter.Prev()) case "seek-ge": @@ -175,6 +183,7 @@ func runInterleavingIterTest(t *testing.T, filename string) { return fmt.Sprintf("unrecognized command %q", td.Cmd) } }) + require.NoError(t, iter.Close()) } type pointIterator struct { diff --git a/internal/keyspan/testdata/interleaving_iter b/internal/keyspan/testdata/interleaving_iter index 9f73e7c08b..b49db93715 100644 --- a/internal/keyspan/testdata/interleaving_iter +++ b/internal/keyspan/testdata/interleaving_iter @@ -239,6 +239,8 @@ iter seek-lt tomato prev seek-lt a +seek-lt tomato +seek-lt tomago ---- -- SpanChanged(nil) -- SpanChanged(q-z:{(#14,RANGEKEYSET,@9,mangos)}) @@ -252,6 +254,16 @@ Span: -- SpanChanged(nil) -- SpanChanged(nil) . +-- SpanChanged(nil) +-- SpanChanged(q-z:{(#14,RANGEKEYSET,@9,mangos)}) +PointKey: q#72057594037927935,21 +Span: q-z:{(#14,RANGEKEYSET,@9,mangos)} +- +-- SpanChanged(nil) +-- SpanChanged(q-z:{(#14,RANGEKEYSET,@9,mangos)}) +PointKey: q#72057594037927935,21 +Span: q-z:{(#14,RANGEKEYSET,@9,mangos)} +- define-rangekeys a-c:{(#10,RANGEKEYSET,@5,apples) (#10,RANGEKEYDEL) (#8,RANGEKEYUNSET,@1) (#4,RANGEKEYSET,@3,bananas) (#4,RANGEKEYSET,@2,oranges)} @@ -906,6 +918,7 @@ seek-prefix-ge b next seek-prefix-ge c next +seek-ge c ---- -- SpanChanged(nil) -- SpanChanged(b-b\x00:{(#5,RANGEKEYSET,@1,foo)}) @@ -923,3 +936,63 @@ Span: c-c\x00:{(#5,RANGEKEYSET,@1,foo)} PointKey: c#8,1 Span: c-c\x00:{(#5,RANGEKEYSET,@1,foo)} - +-- SpanChanged(nil) +-- SpanChanged(b-d:{(#5,RANGEKEYSET,@1,foo)}) +PointKey: c#72057594037927935,21 +Span: b-d:{(#5,RANGEKEYSET,@1,foo)} +- + +# Test NextPrefix + +define-rangekeys +b-e:{(#5,RANGEKEYSET,@9,foo)} +f-g:{(#6,RANGEKEYSET,@9,foo)} +---- +OK + +define-pointkeys +a@4.SET.8 +c@11.SET.8 +c@3.SET.8 +c@1.SET.4 +d@5.SET.3 +e@9.SET.2 +---- +OK + +iter +first +next-prefix +next-prefix +next-prefix +next-prefix +next-prefix +next-prefix +next-prefix +---- +-- SpanChanged(nil) +-- SpanChanged(nil) +PointKey: a@4#8,1 +Span: +- +-- SpanChanged(b-e:{(#5,RANGEKEYSET,@9,foo)}) +PointKey: b#72057594037927935,21 +Span: b-e:{(#5,RANGEKEYSET,@9,foo)} +- +PointKey: c@11#8,1 +Span: b-e:{(#5,RANGEKEYSET,@9,foo)} +- +PointKey: d@5#3,1 +Span: b-e:{(#5,RANGEKEYSET,@9,foo)} +- +-- SpanChanged(nil) +PointKey: e@9#2,1 +Span: +- +-- SpanChanged(f-g:{(#6,RANGEKEYSET,@9,foo)}) +PointKey: f#72057594037927935,21 +Span: f-g:{(#6,RANGEKEYSET,@9,foo)} +- +-- SpanChanged(nil) +. +. diff --git a/internal/keyspan/testdata/interleaving_iter_masking b/internal/keyspan/testdata/interleaving_iter_masking index 9bdffc6c6d..8ad8fb374b 100644 --- a/internal/keyspan/testdata/interleaving_iter_masking +++ b/internal/keyspan/testdata/interleaving_iter_masking @@ -490,7 +490,6 @@ PointKey: a#72057594037927935,21 Span: a-c:{(#1,RANGEKEYSET,@10,apples)} - -- SpanChanged(nil) --- SpanChanged(nil) PointKey: d@9#3,1 Span: - diff --git a/iterator.go b/iterator.go index afe925664f..13b07f564b 100644 --- a/iterator.go +++ b/iterator.go @@ -1762,7 +1762,8 @@ func (i *Iterator) internalNextPrefix(currKeyPrefixLen int) { i.iterKey, i.iterValue = i.iter.NextPrefix(i.prefixOrFullSeekKey) if invariants.Enabled && i.iterKey != nil { if iterKeyPrefixLen := i.split(i.iterKey.UserKey); i.cmp(i.iterKey.UserKey[:iterKeyPrefixLen], i.prefixOrFullSeekKey) < 0 { - panic("pebble: iter.NextPrefix did not advance beyond the current prefix") + panic(errors.AssertionFailedf("pebble: iter.NextPrefix did not advance beyond the current prefix: now at %q; expected to be geq %q", + i.iterKey, i.prefixOrFullSeekKey)) } } } diff --git a/range_keys.go b/range_keys.go index 60fcf620b5..0a46ec1f2e 100644 --- a/range_keys.go +++ b/range_keys.go @@ -516,7 +516,11 @@ func (i *lazyCombinedIter) initCombinedIteration( // Initialize the Iterator's interleaving iterator. i.parent.rangeKey.iiter.Init( &i.parent.comparer, i.parent.pointIter, i.parent.rangeKey.rangeKeyIter, - &i.parent.rangeKeyMasking, i.parent.opts.LowerBound, i.parent.opts.UpperBound) + keyspan.InterleavingIterOpts{ + Mask: &i.parent.rangeKeyMasking, + LowerBound: i.parent.opts.LowerBound, + UpperBound: i.parent.opts.UpperBound, + }) // Set the parent's primary iterator to point to the combined, interleaving // iterator that's now initialized with our current state. diff --git a/scan_internal.go b/scan_internal.go index 13a93bfa62..721181c5cd 100644 --- a/scan_internal.go +++ b/scan_internal.go @@ -802,7 +802,11 @@ func (i *scanInternalIterator) constructPointIter(memtables flushableList, buf * if i.opts.includeObsoleteKeys { iiter := &keyspan.InterleavingIter{} - iiter.Init(i.comparer, &buf.merging, &rangeDelMiter, nil /* mask */, i.opts.LowerBound, i.opts.UpperBound) + iiter.Init(i.comparer, &buf.merging, &rangeDelMiter, + keyspan.InterleavingIterOpts{ + LowerBound: i.opts.LowerBound, + UpperBound: i.opts.UpperBound, + }) i.pointKeyIter = iiter } else { pcIter := &pointCollapsingIterator{ @@ -810,7 +814,10 @@ func (i *scanInternalIterator) constructPointIter(memtables flushableList, buf * merge: i.merge, seqNum: i.seqNum, } - pcIter.iter.Init(i.comparer, &buf.merging, &rangeDelMiter, nil /* mask */, i.opts.LowerBound, i.opts.UpperBound) + pcIter.iter.Init(i.comparer, &buf.merging, &rangeDelMiter, keyspan.InterleavingIterOpts{ + LowerBound: i.opts.LowerBound, + UpperBound: i.opts.UpperBound, + }) i.pointKeyIter = pcIter } i.iter = i.pointKeyIter diff --git a/scan_internal_test.go b/scan_internal_test.go index b6f35f2e6e..a489c753a6 100644 --- a/scan_internal_test.go +++ b/scan_internal_test.go @@ -516,7 +516,7 @@ func TestPointCollapsingIter(t *testing.T) { merge: base.DefaultMerger.Merge, seqNum: math.MaxUint64, } - pcIter.iter.Init(base.DefaultComparer, f, ksIter, nil /* mask */, nil, nil) + pcIter.iter.Init(base.DefaultComparer, f, ksIter, keyspan.InterleavingIterOpts{}) defer pcIter.Close() return runInternalIterCmd(t, d, pcIter, iterCmdVerboseKey)