From 334633b8b0ba2091bcaceab76ce70707ef018bfe Mon Sep 17 00:00:00 2001 From: Cheran Mahalingam Date: Mon, 12 Feb 2024 18:30:02 -0500 Subject: [PATCH] iterator: require prefix filtering in merging iterator Currently for `SeekPrefixGE` calls we filter out keys without a matching prefix at the top layer iterator. This causes the merging iterator to build a larger heap containing entries without the required prefixes. This commit shifts the prefix filtering responsibility to the merging iterator before inserting keys in the heap. Fix #2182. --- db.go | 2 +- error_iter.go | 6 ++ external_iterator.go | 11 +-- get_iter.go | 6 ++ internal.go | 2 + internal/base/iterator.go | 10 ++ internal/invalidating/iter.go | 8 +- iterator.go | 35 +++---- iterator_test.go | 3 +- merging_iter.go | 108 +++++++++++----------- testdata/iter_histories/range_keys_simple | 23 +++++ testdata/iterator | 86 ++++++++++------- testdata/merging_iter | 2 +- testdata/merging_iter_seek | 12 +-- 14 files changed, 190 insertions(+), 124 deletions(-) diff --git a/db.go b/db.go index 045e31bb8a..857ec6886e 100644 --- a/db.go +++ b/db.go @@ -1483,7 +1483,7 @@ func (i *Iterator) constructPointIter( buf.merging.snapshot = i.seqNum buf.merging.batchSnapshot = i.batchSeqNum buf.merging.combinedIterState = &i.lazyCombinedIter.combinedIterState - i.pointIter = invalidating.MaybeWrapIfInvariants(&buf.merging) + i.pointIter = invalidating.MaybeWrapIfInvariants(&buf.merging).(topLevelIterator) i.merging = &buf.merging } diff --git a/error_iter.go b/error_iter.go index 30c442511a..7c003b42a1 100644 --- a/error_iter.go +++ b/error_iter.go @@ -24,6 +24,12 @@ func (c *errorIter) SeekGE(key []byte, flags base.SeekGEFlags) (*InternalKey, ba func (c *errorIter) SeekPrefixGE( prefix, key []byte, flags base.SeekGEFlags, +) (*base.InternalKey, base.LazyValue) { + return c.SeekPrefixGEStrict(prefix, key, flags) +} + +func (c *errorIter) SeekPrefixGEStrict( + prefix, key []byte, flags base.SeekGEFlags, ) (*base.InternalKey, base.LazyValue) { return nil, base.LazyValue{} } diff --git a/external_iterator.go b/external_iterator.go index d53497701c..389b7dd607 100644 --- a/external_iterator.go +++ b/external_iterator.go @@ -166,7 +166,7 @@ func validateExternalIterOpts(iterOpts *IterOptions) error { return nil } -func createExternalPointIter(ctx context.Context, it *Iterator) (internalIterator, error) { +func createExternalPointIter(ctx context.Context, it *Iterator) (topLevelIterator, error) { // TODO(jackson): In some instances we could generate fewer levels by using // L0Sublevels code to organize nonoverlapping files into the same level. // This would allow us to use levelIters and keep a smaller set of data and @@ -236,15 +236,6 @@ func createExternalPointIter(ctx context.Context, it *Iterator) (internalIterato }) } } - if len(mlevels) == 1 && mlevels[0].rangeDelIter == nil { - // Set closePointIterOnce to true. This is because we're bypassing the - // merging iter, which turns Close()s on it idempotent for any child - // iterators. The outer Iterator could call Close() on a point iter twice, - // which sstable iterators do not support (as they release themselves to - // a pool). - it.closePointIterOnce = true - return mlevels[0].iter, nil - } it.alloc.merging.init(&it.opts, &it.stats.InternalStats, it.comparer.Compare, it.comparer.Split, mlevels...) it.alloc.merging.snapshot = base.InternalKeySeqNumMax diff --git a/get_iter.go b/get_iter.go index 036b0e31cf..2b9945e5f8 100644 --- a/get_iter.go +++ b/get_iter.go @@ -54,6 +54,12 @@ func (g *getIter) SeekGE(key []byte, flags base.SeekGEFlags) (*InternalKey, base func (g *getIter) SeekPrefixGE( prefix, key []byte, flags base.SeekGEFlags, +) (*base.InternalKey, base.LazyValue) { + return g.SeekPrefixGEStrict(prefix, key, flags) +} + +func (g *getIter) SeekPrefixGEStrict( + prefix, key []byte, flags base.SeekGEFlags, ) (*base.InternalKey, base.LazyValue) { panic("pebble: SeekPrefixGE unimplemented") } diff --git a/internal.go b/internal.go index f60df91de8..27635bedd6 100644 --- a/internal.go +++ b/internal.go @@ -37,6 +37,8 @@ type InternalKey = base.InternalKey type internalIterator = base.InternalIterator +type topLevelIterator = base.TopLevelIterator + // ErrCorruption is a marker to indicate that data in a file (WAL, MANIFEST, // sstable) isn't in the expected format. var ErrCorruption = base.ErrCorruption diff --git a/internal/base/iterator.go b/internal/base/iterator.go index a0cb2b22ce..e65f60ca59 100644 --- a/internal/base/iterator.go +++ b/internal/base/iterator.go @@ -214,6 +214,16 @@ type InternalIterator interface { fmt.Stringer } +// TopLevelIterator extends InternalIterator to include an additional absolute +// positioning method, SeekPrefixGEStrict. +type TopLevelIterator interface { + InternalIterator + + // SeekPrefixGEStrict extends InternalIterator.SeekPrefixGE with a guarantee + // that the iterator only returns keys matching the prefix. + SeekPrefixGEStrict(prefix, key []byte, flags SeekGEFlags) (*InternalKey, LazyValue) +} + // SeekGEFlags holds flags that may configure the behavior of a forward seek. // Not all flags are relevant to all iterators. type SeekGEFlags uint8 diff --git a/internal/invalidating/iter.go b/internal/invalidating/iter.go index e27db5897d..beb8805ec8 100644 --- a/internal/invalidating/iter.go +++ b/internal/invalidating/iter.go @@ -55,7 +55,7 @@ func IgnoreKinds(kinds ...base.InternalKeyKind) Option { // NewIter constructs a new invalidating iterator that wraps the provided // iterator, trashing buffers for previously returned keys. -func NewIter(originalIterator base.InternalIterator, opts ...Option) base.InternalIterator { +func NewIter(originalIterator base.InternalIterator, opts ...Option) base.TopLevelIterator { i := &iter{iter: originalIterator} for _, opt := range opts { opt.apply(i) @@ -120,6 +120,12 @@ func (i *iter) SeekPrefixGE( return i.update(i.iter.SeekPrefixGE(prefix, key, flags)) } +func (i *iter) SeekPrefixGEStrict( + prefix, key []byte, flags base.SeekGEFlags, +) (*base.InternalKey, base.LazyValue) { + return i.update(i.iter.SeekPrefixGE(prefix, key, flags)) +} + func (i *iter) SeekLT(key []byte, flags base.SeekLTFlags) (*base.InternalKey, base.LazyValue) { return i.update(i.iter.SeekLT(key, flags)) } diff --git a/iterator.go b/iterator.go index a09a3420d0..981cbecab7 100644 --- a/iterator.go +++ b/iterator.go @@ -187,7 +187,7 @@ type Iterator struct { merge Merge comparer base.Comparer iter internalIterator - pointIter internalIterator + pointIter topLevelIterator // Either readState or version is set, but not both. readState *readState version *version @@ -302,16 +302,6 @@ type Iterator struct { batchJustRefreshed bool // batchOnlyIter is set to true for Batch.NewBatchOnlyIter. batchOnlyIter bool - // closePointIterOnce is set to true if this point iter can only be Close()d - // once, _and_ closing i.iter and then i.pointIter would close i.pointIter - // twice. This is necessary to track if the point iter is an internal iterator - // that could release its resources to a pool on Close(), making it harder for - // that iterator to make its own closes idempotent. - // - // TODO(bilal): Update SetOptions to always close out point key iterators when - // they won't be used, so that Close() doesn't need to default to closing - // point iterators twice. - closePointIterOnce bool // Used in some tests to disable the random disabling of seek optimizations. forceEnableSeekOpt bool // Set to true if NextPrefix is not currently permitted. Defaults to false @@ -529,11 +519,19 @@ func (i *Iterator) findNextEntry(limit []byte) { for i.iterKey != nil { key := *i.iterKey - if i.hasPrefix { - if n := i.split(key.UserKey); !i.equal(i.prefixOrFullSeekKey, key.UserKey[:n]) { - return + // The topLevelIterator.StrictSeekPrefixGE contract requires that in + // prefix mode [i.hasPrefix=t], every point key returned by the internal + // iterator must have the current iteration prefix. + if invariants.Enabled && i.hasPrefix { + // Range keys are an exception to the contract and may return a different + // prefix. This case is explicitly handled in the switch statement below. + if key.Kind() != base.InternalKeyKindRangeKeySet { + if n := i.split(key.UserKey); !i.equal(i.prefixOrFullSeekKey, key.UserKey[:n]) { + i.opts.logger.Fatalf("pebble: prefix violation: key %q does not have prefix %q\n", key.UserKey, i.prefixOrFullSeekKey) + } } } + // Compare with limit every time we start at a different user key. // Note that given the best-effort contract of limit, we could avoid a // comparison in the common case by doing this only after @@ -564,6 +562,11 @@ func (i *Iterator) findNextEntry(limit []byte) { switch key.Kind() { case InternalKeyKindRangeKeySet: + if i.hasPrefix { + if n := i.split(key.UserKey); !i.equal(i.prefixOrFullSeekKey, key.UserKey[:n]) { + return + } + } // Save the current key. i.keyBuf = append(i.keyBuf[:0], key.UserKey...) i.key = i.keyBuf @@ -1865,7 +1868,7 @@ func (i *Iterator) nextWithLimit(limit []byte) IterValidityState { i.iterValidityState = IterExhausted return i.iterValidityState } else if i.iterValidityState == IterExhausted { - // No-op, already exhasuted. We avoid executing the Next because it + // No-op, already exhausted. We avoid executing the Next because it // can break invariants: Specifically, a file that fails the bloom // filter test may result in its level being removed from the // merging iterator. The level's removal can cause a lazy combined @@ -2304,7 +2307,7 @@ func (i *Iterator) Close() error { // NB: If the iterators were still connected to i.iter, they may be // closed, but calling Close on a closed internal iterator or fragment // iterator is allowed. - if i.pointIter != nil && !i.closePointIterOnce { + if i.pointIter != nil { i.err = firstError(i.err, i.pointIter.Close()) } if i.rangeKey != nil && i.rangeKey.rangeKeyIter != nil { diff --git a/iterator_test.go b/iterator_test.go index 8e77c8012b..026e959e6f 100644 --- a/iterator_test.go +++ b/iterator_test.go @@ -931,6 +931,7 @@ func TestIteratorSeekOpt(t *testing.T) { } return iters, err } + d.opts.Comparer.Split = func(a []byte) int { return len(a) } return s case "iter": @@ -944,7 +945,7 @@ func TestIteratorSeekOpt(t *testing.T) { } iter, _ = snap.NewIter(nil) iter.readSampling.forceReadSampling = true - iter.comparer.Split = func(a []byte) int { return len(a) } + iter.comparer.Split = d.opts.Comparer.Split iter.forceEnableSeekOpt = true iter.merging.forceEnableSeekOpt = true } diff --git a/merging_iter.go b/merging_iter.go index d1028eb95b..741d7687dd 100644 --- a/merging_iter.go +++ b/merging_iter.go @@ -603,31 +603,8 @@ func (m *mergingIter) switchToMaxHeap() error { return m.initMaxHeap() } -// maybeNextEntryWithinPrefix steps to the next entry, as long as the iteration -// prefix has not already been exceeded. If it has, it exhausts the iterator by -// resetting the heap to empty. -func (m *mergingIter) maybeNextEntryWithinPrefix(l *mergingIterLevel) error { - if s := m.split(l.iterKey.UserKey); !bytes.Equal(m.prefix, l.iterKey.UserKey[:s]) { - // The item at the root of the heap already exceeds the iteration - // prefix. We should not advance any more. Clear the heap to reflect - // that the iterator is now exhausted (within this prefix, at - // least). - m.heap.items = m.heap.items[:0] - return nil - } - return m.nextEntry(l, nil /* succKey */) -} - // nextEntry unconditionally steps to the next entry. item is the current top // item in the heap. -// -// nextEntry should be called directly when not in prefix-iteration mode, or by -// Next. During prefix iteration mode, all other callers should use -// maybeNextEntryWithinPrefix which will avoid advancing the iterator if the -// current iteration prefix has been exhausted. See the comment within -// nextEntry's body for an explanation of why other callers should call -// maybeNextEntryWithinPrefix, which will ensure the documented invariant is -// preserved. func (m *mergingIter) nextEntry(l *mergingIterLevel, succKey []byte) error { // INVARIANT: If in prefix iteration mode, item.iterKey must have a prefix equal // to m.prefix. This invariant is important for ensuring TrySeekUsingNext @@ -665,8 +642,19 @@ func (m *mergingIter) nextEntry(l *mergingIterLevel, succKey []byte) error { l.iterKey, l.iterValue = l.iter.NextPrefix(succKey) } - if l.iterKey != nil { - if m.heap.len() > 1 { + if l.iterKey == nil { + if err := l.iter.Error(); err != nil { + return err + } + m.heap.pop() + } else { + if m.prefix != nil && !bytes.Equal(m.prefix, l.iterKey.UserKey[:m.split(l.iterKey.UserKey)]) { + // Set keys without a matching prefix to their zero values when in prefix + // iteration mode and remove iterated level from heap. + l.iterKey = nil + l.iterValue = base.LazyValue{} + m.heap.pop() + } else if m.heap.len() > 1 { m.heap.fix(0) } if l.rangeDelIter != oldRangeDelIter { @@ -674,11 +662,6 @@ func (m *mergingIter) nextEntry(l *mergingIterLevel, succKey []byte) error { // next sstable. We have to update the tombstone for oldTopLevel as well. oldTopLevel-- } - } else { - if err := l.iter.Error(); err != nil { - return err - } - m.heap.pop() } // The cached tombstones are only valid for the levels @@ -797,13 +780,7 @@ func (m *mergingIter) isNextEntryDeleted(item *mergingIterLevel) (bool, error) { return true, nil } if l.tombstone.CoversAt(m.snapshot, item.iterKey.SeqNum()) { - var err error - if m.prefix == nil { - err = m.nextEntry(item, nil /* succKey */) - } else { - err = m.maybeNextEntryWithinPrefix(item) - } - if err != nil { + if err := m.nextEntry(item, nil /* succKey */); err != nil { return false, err } return true, nil @@ -830,11 +807,7 @@ func (m *mergingIter) findNextEntry() (*InternalKey, base.LazyValue) { // keep sstables open until we've surpassed their end boundaries so that // their range deletions are visible. if m.levels[item.index].isIgnorableBoundaryKey { - if m.prefix == nil { - m.err = m.nextEntry(item, nil /* succKey */) - } else { - m.err = m.maybeNextEntryWithinPrefix(item) - } + m.err = m.nextEntry(item, nil /* succKey */) if m.err != nil { return nil, base.LazyValue{} } @@ -855,11 +828,7 @@ func (m *mergingIter) findNextEntry() (*InternalKey, base.LazyValue) { // Check if the key is visible at the iterator sequence numbers. if !item.iterKey.Visible(m.snapshot, m.batchSnapshot) { - if m.prefix == nil { - m.err = m.nextEntry(item, nil /* succKey */) - } else { - m.err = m.maybeNextEntryWithinPrefix(item) - } + m.err = m.nextEntry(item, nil /* succKey */) if m.err != nil { return nil, base.LazyValue{} } @@ -1064,6 +1033,14 @@ func (m *mergingIter) seekGE(key []byte, level int, flags base.SeekGEFlags) erro l := &m.levels[level] if m.prefix != nil { l.iterKey, l.iterValue = l.iter.SeekPrefixGE(m.prefix, key, flags) + if l.iterKey != nil { + if n := m.split(l.iterKey.UserKey); !bytes.Equal(m.prefix, l.iterKey.UserKey[:n]) { + // Prevent keys without a matching prefix from being added to the heap by setting + // iterKey and iterValue to their zero values before calling initMinHeap. + l.iterKey = nil + l.iterValue = base.LazyValue{} + } + } } else { l.iterKey, l.iterValue = l.iter.SeekGE(key, flags) } @@ -1120,18 +1097,31 @@ func (m *mergingIter) SeekGE(key []byte, flags base.SeekGEFlags) (*InternalKey, return m.findNextEntry() } -// SeekPrefixGE implements base.InternalIterator.SeekPrefixGE. Note that -// SeekPrefixGE only checks the upper bound. It is up to the caller to ensure -// that key is greater than or equal to the lower bound. +// SeekPrefixGE implements base.InternalIterator.SeekPrefixGE. func (m *mergingIter) SeekPrefixGE( prefix, key []byte, flags base.SeekGEFlags, +) (*base.InternalKey, base.LazyValue) { + return m.SeekPrefixGEStrict(prefix, key, flags) +} + +// SeekPrefixGEStrict implements topLevelIterator.SeekPrefixGEStrict. Note that +// SeekPrefixGEStrict explicitly checks that the key has a matching prefix. +func (m *mergingIter) SeekPrefixGEStrict( + prefix, key []byte, flags base.SeekGEFlags, ) (*base.InternalKey, base.LazyValue) { m.prefix = prefix m.err = m.seekGE(key, 0 /* start level */, flags) if m.err != nil { return nil, base.LazyValue{} } - return m.findNextEntry() + + iterKey, iterValue := m.findNextEntry() + if invariants.Enabled && iterKey != nil { + if n := m.split(iterKey.UserKey); !bytes.Equal(m.prefix, iterKey.UserKey[:n]) { + m.logger.Fatalf("mergingIter: prefix violation: returning key %q without prefix %q\n", iterKey, m.prefix) + } + } + return iterKey, iterValue } // Seeks levels >= level to < key. Additionally uses range tombstones to extend the seeks. @@ -1258,14 +1248,20 @@ func (m *mergingIter) Next() (*InternalKey, base.LazyValue) { } // NB: It's okay to call nextEntry directly even during prefix iteration - // mode (as opposed to indirectly through maybeNextEntryWithinPrefix). - // During prefix iteration mode, we rely on the caller to not call Next if - // the iterator has already advanced beyond the iteration prefix. See the - // comment above the base.InternalIterator interface. + // mode. During prefix iteration mode, we rely on the caller to not call + // Next if the iterator has already advanced beyond the iteration prefix. + // See the comment above the base.InternalIterator interface. if m.err = m.nextEntry(m.heap.items[0], nil /* succKey */); m.err != nil { return nil, base.LazyValue{} } - return m.findNextEntry() + + iterKey, iterValue := m.findNextEntry() + if invariants.Enabled && m.prefix != nil && iterKey != nil { + if n := m.split(iterKey.UserKey); !bytes.Equal(m.prefix, iterKey.UserKey[:n]) { + m.logger.Fatalf("mergingIter: prefix violation: returning key %q without prefix %q\n", iterKey, m.prefix) + } + } + return iterKey, iterValue } func (m *mergingIter) NextPrefix(succKey []byte) (*InternalKey, LazyValue) { diff --git a/testdata/iter_histories/range_keys_simple b/testdata/iter_histories/range_keys_simple index e65712d5e7..c303a02298 100644 --- a/testdata/iter_histories/range_keys_simple +++ b/testdata/iter_histories/range_keys_simple @@ -486,3 +486,26 @@ b: (., [b-d) @2=foo UPDATED) b: (., [b-d) @2=foo UPDATED) b: (., [b-d) @2=foo) . + +# Test Next-ing while in prefix iteration mode when using the keyspan iterator prefix +# seek optimization (after a SeekGE(a) the keyspan iterator can avoid reseeking for +# a SeekPrefixGE if the prefix falls within the previous span's bounds). + +reset +---- + +batch commit +set a a +range-key-set a b @1 foo +range-key-set b d @2 bar +---- +committed 3 keys + +combined-iter +seek-ge a +seek-prefix-ge a +next +---- +a: (a, [a-b) @1=foo UPDATED) +a: (a, [a-"a\x00") @1=foo UPDATED) +. diff --git a/testdata/iterator b/testdata/iterator index cad4f0d2f2..17c4a76b67 100644 --- a/testdata/iterator +++ b/testdata/iterator @@ -1,3 +1,16 @@ +define +a.DEL.2: +a.SET.1:b +b.SET.3:c +---- + +iter seq=4 +seek-prefix-ge a +---- +. +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 1B, tombstoned 0))) + define a.SET.1:b ---- @@ -75,7 +88,6 @@ a: (c, .) stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 2B, tombstoned 0))) - define a.DEL.2: a.SET.1:b @@ -126,8 +138,7 @@ iter seq=2 seek-prefix-ge 1 ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 0B, tombstoned 0))) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) define a.DEL.2: @@ -163,14 +174,14 @@ seek-prefix-ge a ---- . stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 2B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 1B, tombstoned 0))) iter seq=3 seek-prefix-ge a ---- . stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 2B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 1B, tombstoned 0))) iter seq=2 seek-prefix-ge a @@ -205,7 +216,7 @@ seek-prefix-ge c . c: (d, .) stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 4), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 7, key-bytes 7B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 5, key-bytes 5B, value-bytes 3B, tombstoned 0))) iter seq=3 seek-prefix-ge a @@ -317,7 +328,7 @@ next a: (a, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 2B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=4 seek-prefix-ge b @@ -326,7 +337,7 @@ next b: (b, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 2B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=4 @@ -414,7 +425,7 @@ next a: (b, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 2B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=2 seek-prefix-ge b @@ -438,7 +449,7 @@ next a: (a, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 3B, value-bytes 3B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=5 seek-prefix-ge a @@ -447,7 +458,7 @@ next a: (a, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 3B, value-bytes 3B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=5 seek-prefix-ge aa @@ -463,7 +474,7 @@ next aa: (aa, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 5B, value-bytes 5B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 2B, value-bytes 2B, tombstoned 0))) iter seq=5 seek-prefix-ge aa @@ -472,7 +483,7 @@ next aa: (aa, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 5B, value-bytes 5B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 2B, value-bytes 2B, tombstoned 0))) iter seq=5 seek-prefix-ge aaa @@ -481,7 +492,7 @@ next aaa: (aaa, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 4B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=5 seek-prefix-ge aaa @@ -618,7 +629,7 @@ a: (a, .) aa: (aa, .) . stats: (interface (dir, seek, step): (fwd, 5, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 5, 0), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 7, key-bytes 12B, value-bytes 12B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 5, key-bytes 10B, value-bytes 10B, tombstoned 0))) define bb.DEL.2: @@ -631,7 +642,7 @@ seek-prefix-ge bb ---- . stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 7B, value-bytes 2B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 4B, value-bytes 1B, tombstoned 0))) define @@ -789,7 +800,7 @@ next a: (bc, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 4B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=2 seek-prefix-ge a @@ -798,7 +809,7 @@ next a: (b, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 4B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=4 seek-prefix-ge a @@ -807,7 +818,7 @@ next a: (bcd, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 4B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=2 seek-prefix-ge a @@ -816,7 +827,7 @@ next a: (b, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 4B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=3 seek-prefix-ge a @@ -825,7 +836,7 @@ next a: (bc, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 4B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=3 seek-prefix-ge c @@ -837,8 +848,7 @@ iter seq=3 seek-prefix-ge 1 ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) iter seq=3 seek-prefix-ge a @@ -867,7 +877,7 @@ a: (bc, .) . . stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 5B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=2 seek-prefix-ge a @@ -878,7 +888,7 @@ a: (b, .) . . stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 5B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=4 seek-prefix-ge a @@ -887,7 +897,7 @@ next a: (bcd, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 5B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=2 seek-prefix-ge a @@ -896,7 +906,7 @@ next a: (b, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 4, key-bytes 5B, value-bytes 4B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 3B, value-bytes 3B, tombstoned 0))) iter seq=3 seek-prefix-ge aa @@ -905,14 +915,14 @@ next aa: (ab, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 5B, value-bytes 3B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 4B, value-bytes 2B, tombstoned 0))) iter seq=4 seek-prefix-ge aa ---- aa: (ab, .) stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 3, key-bytes 5B, value-bytes 3B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 4B, value-bytes 2B, tombstoned 0))) define a.SET.1:a @@ -1322,7 +1332,7 @@ next a: (a, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 3B, value-bytes 3B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=2 lower=a upper=b seek-prefix-ge a @@ -1331,7 +1341,7 @@ next a: (a, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 3B, value-bytes 3B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=2 lower=a upper=c seek-prefix-ge a @@ -1340,7 +1350,7 @@ next a: (a, .) . stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), -(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 3B, value-bytes 3B, tombstoned 0))) +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) iter seq=2 lower=a upper=aaa seek-prefix-ge aa @@ -1374,6 +1384,18 @@ b: (b, .) stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 2, key-bytes 2B, value-bytes 2B, tombstoned 0))) +define +a.SET.2:a +b.SET.1:b +---- + +iter seq=2 +seek-prefix-ge a +---- +. +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0B, cached 0B, read-time 0s)), (points: (count 1, key-bytes 1B, value-bytes 1B, tombstoned 0))) + define a.SINGLEDEL.1: ---- diff --git a/testdata/merging_iter b/testdata/merging_iter index 1a84fbd0ac..e4b2155f7f 100644 --- a/testdata/merging_iter +++ b/testdata/merging_iter @@ -591,7 +591,7 @@ seek-prefix-ge d true a#10,SET:a10 . . -d#10,SET:d10 +. d#10,SET:d10 iter diff --git a/testdata/merging_iter_seek b/testdata/merging_iter_seek index 2205da2cbb..57150efae5 100644 --- a/testdata/merging_iter_seek +++ b/testdata/merging_iter_seek @@ -73,7 +73,7 @@ seek-prefix-ge a0 next ---- a0:0 -a1:1 +. iter seek-prefix-ge a0 @@ -187,9 +187,9 @@ seek-prefix-ge aaa next ---- a:0 -aa:1 +. aaa:2 -b:3 +. iter seek-prefix-ge aa @@ -204,7 +204,7 @@ next prev ---- aa:1 -aaa:2 +. err=pebble: unsupported reverse prefix iteration iter @@ -213,7 +213,7 @@ next prev ---- aa:1 -aaa:2 +. err=pebble: unsupported reverse prefix iteration iter @@ -221,7 +221,7 @@ seek-prefix-ge aaa next ---- aaa:2 -b:3 +. iter seek-prefix-ge aaa