From 9494dd393a8396959c79be204428667a2af64e17 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Sun, 3 Apr 2022 13:34:39 +0000 Subject: [PATCH] storage: remove `MVCCIterator.SetUpperBound` and tweak `ClearIterRange` This patch removes the `MVCCIterator.SetUpperBound()` method, which was only used by `MVCCIterator.ClearIterRange()`. `ClearIterRange` now also creates the iterator itself, instead of having it passed in. This ensures the iterator is created with appropriate bounds and options, which becomes even more important with the introduction of range keys. If further flexibility is needed, we can consider passing an `IterOptions` struct instead of a key span. Release note: None --- pkg/kv/kvserver/abortspan/abortspan.go | 8 +- pkg/kv/kvserver/batch_spanset_test.go | 12 +- pkg/kv/kvserver/batcheval/cmd_clear_range.go | 7 +- .../batcheval/cmd_clear_range_test.go | 4 +- pkg/kv/kvserver/spanset/batch.go | 14 +- pkg/storage/bench_pebble_test.go | 4 +- pkg/storage/bench_test.go | 21 +-- pkg/storage/engine.go | 44 +----- pkg/storage/engine_test.go | 4 +- pkg/storage/intent_interleaving_iter.go | 13 +- pkg/storage/intent_interleaving_iter_test.go | 28 +--- pkg/storage/pebble.go | 8 +- pkg/storage/pebble_batch.go | 27 ++-- pkg/storage/pebble_iterator.go | 45 +----- pkg/storage/pebble_test.go | 135 ------------------ pkg/storage/sst_writer.go | 2 +- .../testdata/intent_interleaving_iter/basic | 6 +- 17 files changed, 50 insertions(+), 332 deletions(-) diff --git a/pkg/kv/kvserver/abortspan/abortspan.go b/pkg/kv/kvserver/abortspan/abortspan.go index 8ecc24c9ef4f..55b020b6675e 100644 --- a/pkg/kv/kvserver/abortspan/abortspan.go +++ b/pkg/kv/kvserver/abortspan/abortspan.go @@ -76,13 +76,9 @@ func (sc *AbortSpan) max() roachpb.Key { // ClearData removes all persisted items stored in the cache. func (sc *AbortSpan) ClearData(e storage.Engine) error { - // NB: The abort span is a Range-ID local key which has no versions or intents. - iter := e.NewMVCCIterator(storage.MVCCKeyIterKind, storage.IterOptions{UpperBound: sc.max()}) - defer iter.Close() - b := e.NewUnindexedBatch(true /* writeOnly */) + b := e.NewUnindexedBatch(false /* writeOnly */) defer b.Close() - err := b.ClearIterRange(iter, sc.min(), sc.max()) - if err != nil { + if err := b.ClearIterRange(sc.min(), sc.max()); err != nil { return err } return b.Commit(false /* sync */) diff --git a/pkg/kv/kvserver/batch_spanset_test.go b/pkg/kv/kvserver/batch_spanset_test.go index d7d9a4e6ea3c..8358853170b6 100644 --- a/pkg/kv/kvserver/batch_spanset_test.go +++ b/pkg/kv/kvserver/batch_spanset_test.go @@ -78,9 +78,7 @@ func TestSpanSetBatchBoundaries(t *testing.T) { t.Errorf("ClearRange: unexpected error %v", err) } { - iter := batch.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: roachpb.KeyMax}) - err := batch.ClearIterRange(iter, outsideKey.Key, outsideKey2.Key) - iter.Close() + err := batch.ClearIterRange(outsideKey.Key, outsideKey2.Key) if !isWriteSpanErr(err) { t.Errorf("ClearIterRange: unexpected error %v", err) } @@ -101,9 +99,7 @@ func TestSpanSetBatchBoundaries(t *testing.T) { t.Errorf("ClearRange: unexpected error %v", err) } { - iter := batch.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: roachpb.KeyMax}) - err := batch.ClearIterRange(iter, insideKey2.Key, outsideKey4.Key) - iter.Close() + err := batch.ClearIterRange(outsideKey2.Key, outsideKey4.Key) if !isWriteSpanErr(err) { t.Errorf("ClearIterRange: unexpected error %v", err) } @@ -325,9 +321,7 @@ func TestSpanSetBatchTimestamps(t *testing.T) { t.Errorf("Clear: unexpected error %v", err) } { - iter := batch.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{UpperBound: roachpb.KeyMax}) - err := batch.ClearIterRange(iter, wkey.Key, wkey.Key) - iter.Close() + err := batch.ClearIterRange(wkey.Key, wkey.Key) if !isWriteSpanErr(err) { t.Errorf("ClearIterRange: unexpected error %v", err) } diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range.go b/pkg/kv/kvserver/batcheval/cmd_clear_range.go index 05e22b5fd9e9..41755777c690 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range.go @@ -116,12 +116,7 @@ func ClearRange( if statsDelta.ContainsEstimates == 0 && statsDelta.Total() < ClearRangeBytesThreshold { log.VEventf(ctx, 2, "delta=%d < threshold=%d; using non-range clear", statsDelta.Total(), ClearRangeBytesThreshold) - iter := readWriter.NewMVCCIterator(storage.MVCCKeyAndIntentsIterKind, storage.IterOptions{ - LowerBound: from, - UpperBound: to, - }) - defer iter.Close() - if err = readWriter.ClearIterRange(iter, from, to); err != nil { + if err = readWriter.ClearIterRange(from, to); err != nil { return result.Result{}, err } return pd, nil diff --git a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go index c31a23aa22ed..45ab490327f1 100644 --- a/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_clear_range_test.go @@ -35,9 +35,9 @@ type wrappedBatch struct { clearRangeCount int } -func (wb *wrappedBatch) ClearIterRange(iter storage.MVCCIterator, start, end roachpb.Key) error { +func (wb *wrappedBatch) ClearIterRange(start, end roachpb.Key) error { wb.clearIterCount++ - return wb.Batch.ClearIterRange(iter, start, end) + return wb.Batch.ClearIterRange(start, end) } func (wb *wrappedBatch) ClearMVCCRangeAndIntents(start, end roachpb.Key) error { diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index ae012cb87a87..109c045bacd4 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -208,11 +208,6 @@ func (i *MVCCIterator) FindSplitKey( return i.i.FindSplitKey(start, end, minSplitKey, targetSize) } -// SetUpperBound is part of the storage.MVCCIterator interface. -func (i *MVCCIterator) SetUpperBound(key roachpb.Key) { - i.i.SetUpperBound(key) -} - // Stats is part of the storage.MVCCIterator interface. func (i *MVCCIterator) Stats() storage.IteratorStats { return i.i.Stats() @@ -374,11 +369,6 @@ func (i *EngineIterator) UnsafeRawEngineKey() []byte { return i.i.UnsafeRawEngineKey() } -// SetUpperBound is part of the storage.EngineIterator interface. -func (i *EngineIterator) SetUpperBound(key roachpb.Key) { - i.i.SetUpperBound(key) -} - // GetRawIter is part of the storage.EngineIterator interface. func (i *EngineIterator) GetRawIter() *pebble.Iterator { return i.i.GetRawIter() @@ -592,11 +582,11 @@ func (s spanSetWriter) ClearMVCCRange(start, end storage.MVCCKey) error { return s.w.ClearMVCCRange(start, end) } -func (s spanSetWriter) ClearIterRange(iter storage.MVCCIterator, start, end roachpb.Key) error { +func (s spanSetWriter) ClearIterRange(start, end roachpb.Key) error { if err := s.checkAllowedRange(start, end); err != nil { return err } - return s.w.ClearIterRange(iter, start, end) + return s.w.ClearIterRange(start, end) } func (s spanSetWriter) Merge(key storage.MVCCKey, value []byte) error { diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 7a540849a7c2..5d9d3eee3fdc 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -340,9 +340,7 @@ func BenchmarkClearMVCCRange_Pebble(b *testing.B) { func BenchmarkClearIterRange_Pebble(b *testing.B) { ctx := context.Background() runClearRange(ctx, b, setupMVCCPebble, func(eng Engine, batch Batch, start, end MVCCKey) error { - iter := eng.NewMVCCIterator(MVCCKeyIterKind, IterOptions{UpperBound: roachpb.KeyMax}) - defer iter.Close() - return batch.ClearIterRange(iter, start.Key, end.Key) + return batch.ClearIterRange(start.Key, end.Key) }) } diff --git a/pkg/storage/bench_test.go b/pkg/storage/bench_test.go index 4383b17c1a56..aebab38308bb 100644 --- a/pkg/storage/bench_test.go +++ b/pkg/storage/bench_test.go @@ -1286,29 +1286,12 @@ func runClearRange( }) defer eng.Close() - // It is not currently possible to ClearRange(NilKey, MVCCKeyMax) thanks to a - // variety of hacks inside of ClearRange that explode if provided the NilKey. - // So instead we start our ClearRange at the first key that actually exists. - // - // TODO(benesch): when those hacks are removed, don't bother computing the - // first key and simply ClearRange(NilKey, MVCCKeyMax). - // - // TODO(sumeer): we are now seeking starting at LocalMax, so the - // aforementioned issue is probably resolved. Clean this up. - iter := eng.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) - defer iter.Close() - iter.SeekGE(MVCCKey{Key: keys.LocalMax}) - if ok, err := iter.Valid(); !ok { - b.Fatalf("unable to find first key (err: %v)", err) - } - firstKey := iter.Key() - b.SetBytes(rangeBytes) b.ResetTimer() for i := 0; i < b.N; i++ { - batch := eng.NewUnindexedBatch(true /* writeOnly */) - if err := clearRange(eng, batch, firstKey, MVCCKeyMax); err != nil { + batch := eng.NewUnindexedBatch(false /* writeOnly */) + if err := clearRange(eng, batch, MVCCKey{Key: keys.LocalMax}, MVCCKeyMax); err != nil { b.Fatal(err) } // NB: We don't actually commit the batch here as we don't want to delete diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 61780523cec9..7cfac155bb58 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -169,26 +169,6 @@ type MVCCIterator interface { // package-level MVCCFindSplitKey instead. For correct operation, the caller // must set the upper bound on the iterator before calling this method. FindSplitKey(start, end, minSplitKey roachpb.Key, targetSize int64) (MVCCKey, error) - // SetUpperBound installs a new upper bound for this iterator. The caller - // can modify the parameter after this function returns. This must not be a - // nil key. When Reader.ConsistentIterators is true, prefer creating a new - // iterator. - // - // Due to the rare use, we are limiting this method to not switch an - // iterator from a global key upper-bound to a local key upper-bound (it - // simplifies some code in intentInterleavingIter) or vice versa. Iterator - // reuse already happens under-the-covers for most Reader implementations - // when constructing a new iterator, and that is a much cleaner solution. - // - // TODO(sumeer): this method is rarely used and is a source of complexity - // since intentInterleavingIter needs to fiddle with the bounds of its - // underlying iterators when this is called. Currently only used by - // pebbleBatch.ClearIterRange to modify the upper bound of the iterator it - // is given: this use is unprincipled and there is a comment in that code - // about it. The caller is already usually setting the bounds accurately, - // and in some cases the callee is tightening the upper bound. Remove that - // use case and remove this from the interface. - SetUpperBound(roachpb.Key) // Stats returns statistics about the iterator. Stats() IteratorStats // SupportsPrev returns true if MVCCIterator implementation supports reverse @@ -239,10 +219,6 @@ type EngineIterator interface { // Value returns the current value as a byte slice. // REQUIRES: latest positioning function returned valid=true. Value() []byte - // SetUpperBound installs a new upper bound for this iterator. When - // Reader.ConsistentIterators is true, prefer creating a new iterator. - // TODO(sumeer): remove this method. - SetUpperBound(roachpb.Key) // GetRawIter is a low-level method only for use in the storage package, // that returns the underlying pebble Iterator. GetRawIter() *pebble.Iterator @@ -326,8 +302,8 @@ const ( // Specifically: // - If both bounds are set they must not span from local to global. // - Any bound (lower or upper), constrains the iterator for its lifetime to - // one of local or global keys. The iterator will not tolerate a seek or - // SetUpperBound call that violates this constraint. + // one of local or global keys. The iterator will not tolerate a seek that + // violates this constraint. // We could, with significant code complexity, not constrain an iterator for // its lifetime, and allow a seek that specifies a global (local) key to // change the constraint to global (local). This would allow reuse of the @@ -563,17 +539,11 @@ type Writer interface { // It is safe to modify the contents of the arguments after it returns. ClearMVCCRange(start, end MVCCKey) error - // ClearIterRange removes a set of entries, from start (inclusive) to end - // (exclusive). Similar to Clear and ClearRange, this method actually - // removes entries from the storage engine. Unlike ClearRange, the entries - // to remove are determined by iterating over iter and per-key storage - // tombstones (not MVCC tombstones) are generated. If the MVCCIterator was - // constructed using MVCCKeyAndIntentsIterKind, any separated intents/locks - // will also be cleared. - // - // It is safe to modify the contents of the arguments after ClearIterRange - // returns. - ClearIterRange(iter MVCCIterator, start, end roachpb.Key) error + // ClearIterRange removes all keys in the given span using an iterator to + // iterate over point keys and remove them from the storage engine using + // per-key storage tombstones (not MVCC tombstones). Any separated + // intents/locks will also be cleared. + ClearIterRange(start, end roachpb.Key) error // Merge is a high-performance write operation used for values which are // accumulated over several writes. Multiple values can be merged diff --git a/pkg/storage/engine_test.go b/pkg/storage/engine_test.go index b917225ec2af..68df6fb41245 100644 --- a/pkg/storage/engine_test.go +++ b/pkg/storage/engine_test.go @@ -1013,9 +1013,7 @@ func TestEngineDeleteIterRange(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) testEngineDeleteRange(t, func(engine Engine, start, end MVCCKey) error { - iter := engine.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{UpperBound: roachpb.KeyMax}) - defer iter.Close() - return engine.ClearIterRange(iter, start.Key, end.Key) + return engine.ClearIterRange(start.Key, end.Key) }) } diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index 07ec2b1d4e6e..2be678c0fae3 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -80,7 +80,7 @@ const ( // bounds, the call to newIntentInterleavingIter must have specified at least // one of the lower or upper bound. We use that to "constrain" the iterator as // either a local key iterator or global key iterator and panic if a caller -// violates that in a subsequent SeekGE/SeekLT/SetUpperBound call. +// violates that in a subsequent SeekGE/SeekLT call. type intentInterleavingIter struct { prefix bool constraint intentInterleavingIterConstraint @@ -954,17 +954,6 @@ func (i *intentInterleavingIter) FindSplitKey( return findSplitKeyUsingIterator(i, start, end, minSplitKey, targetSize) } -func (i *intentInterleavingIter) SetUpperBound(key roachpb.Key) { - i.iter.SetUpperBound(key) - // Preceding call to SetUpperBound has confirmed that key != nil. - if i.constraint != notConstrained { - i.checkConstraint(key, true) - } - var intentUpperBound roachpb.Key - intentUpperBound, i.intentKeyBuf = keys.LockTableSingleKey(key, i.intentKeyBuf) - i.intentIter.SetUpperBound(intentUpperBound) -} - func (i *intentInterleavingIter) Stats() IteratorStats { stats := i.iter.Stats() intentStats := i.intentIter.Stats() diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index cd586706b286..b09a37dcaecc 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -184,7 +184,7 @@ func checkAndOutputIter(iter MVCCIterator, b *strings.Builder) { // - iter: for iterating, is defined as // iter [lower=] [upper=] [prefix=] // followed by newline separated sequence of operations: -// next, prev, seek-lt, seek-ge, set-upper, next-key, stats +// next, prev, seek-lt, seek-ge, next-key, stats // // Keys are interpreted as: // - starting with L is interpreted as a local-range key. @@ -362,10 +362,6 @@ func TestIntentInterleavingIter(t *testing.T) { iter.NextKey() fmt.Fprintf(&b, "next-key: ") checkAndOutputIter(iter, &b) - case "set-upper": - k := scanRoachKey(t, d, "k") - iter.SetUpperBound(k) - fmt.Fprintf(&b, "set-upper %s\n", string(makePrintableKey(MVCCKey{Key: k}).Key)) case "stats": stats := iter.Stats() // Setting non-deterministic InternalStats to empty. @@ -395,8 +391,6 @@ func TestIntentInterleavingIterBoundaries(t *testing.T) { iter := newIntentInterleavingIterator(eng, opts).(*intentInterleavingIter) defer iter.Close() require.Equal(t, constrainedToLocal, iter.constraint) - iter.SetUpperBound(keys.LocalMax) - require.Equal(t, constrainedToLocal, iter.constraint) iter.SeekLT(MVCCKey{Key: keys.LocalMax}) }() func() { @@ -404,8 +398,6 @@ func TestIntentInterleavingIterBoundaries(t *testing.T) { iter := newIntentInterleavingIterator(eng, opts).(*intentInterleavingIter) defer iter.Close() require.Equal(t, constrainedToLocal, iter.constraint) - iter.SetUpperBound(keys.LocalMax) - require.Equal(t, constrainedToLocal, iter.constraint) }() require.Panics(t, func() { opts := IterOptions{UpperBound: keys.LocalMax} @@ -420,13 +412,6 @@ func TestIntentInterleavingIterBoundaries(t *testing.T) { defer iter.Close() require.Equal(t, constrainedToGlobal, iter.constraint) }() - require.Panics(t, func() { - opts := IterOptions{LowerBound: keys.LocalMax} - iter := newIntentInterleavingIterator(eng, opts).(*intentInterleavingIter) - defer iter.Close() - require.Equal(t, constrainedToGlobal, iter.constraint) - iter.SetUpperBound(keys.LocalMax) - }) func() { opts := IterOptions{LowerBound: keys.LocalMax} iter := newIntentInterleavingIterator(eng, opts).(*intentInterleavingIter) @@ -923,14 +908,12 @@ func BenchmarkIntentInterleavingSeekGEAndIter(b *testing.B) { iter = state.eng.NewMVCCIterator(MVCCKeyIterKind, opts) } b.ResetTimer() - var unsafeKey MVCCKey for i := 0; i < b.N; i++ { j := i % len(seekKeys) upperIndex := j + 1 + scanTo := MVCCKey{Key: endKey} if upperIndex < len(seekKeys) { - iter.SetUpperBound(seekKeys[upperIndex]) - } else { - iter.SetUpperBound(endKey) + scanTo.Key = seekKeys[upperIndex] } iter.SeekGE(MVCCKey{Key: seekKeys[j]}) for { @@ -941,11 +924,12 @@ func BenchmarkIntentInterleavingSeekGEAndIter(b *testing.B) { if !valid { break } - unsafeKey = iter.UnsafeKey() + if iter.UnsafeKey().Compare(scanTo) >= 0 { + break + } iter.Next() } } - _ = unsafeKey }) } }) diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 344754045f76..ef126ed268ce 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1225,12 +1225,12 @@ func (p *Pebble) clearRange(start, end MVCCKey) error { } // ClearIterRange implements the Engine interface. -func (p *Pebble) ClearIterRange(iter MVCCIterator, start, end roachpb.Key) error { +func (p *Pebble) ClearIterRange(start, end roachpb.Key) error { // Write all the tombstones in one batch. - batch := p.NewUnindexedBatch(true /* writeOnly */) + batch := p.NewUnindexedBatch(false /* writeOnly */) defer batch.Close() - if err := batch.ClearIterRange(iter, start, end); err != nil { + if err := batch.ClearIterRange(start, end); err != nil { return err } return batch.Commit(true) @@ -2066,7 +2066,7 @@ func (p *pebbleReadOnly) ClearMVCCRange(start, end MVCCKey) error { panic("not implemented") } -func (p *pebbleReadOnly) ClearIterRange(iter MVCCIterator, start, end roachpb.Key) error { +func (p *pebbleReadOnly) ClearIterRange(start, end roachpb.Key) error { panic("not implemented") } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index ffb853472074..cd20b3cdf5ca 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -400,27 +400,24 @@ func (p *pebbleBatch) clearRange(start, end MVCCKey) error { return p.batch.DeleteRange(p.buf, buf2, nil) } -// Clear implements the Batch interface. -func (p *pebbleBatch) ClearIterRange(iter MVCCIterator, start, end roachpb.Key) error { - // Note that this method has the side effect of modifying iter's bounds. - // Since all calls to `ClearIterRange` are on new throwaway iterators with no - // lower bounds, calling SetUpperBound should be sufficient and safe. - // Furthermore, the start and end keys are always metadata keys (i.e. - // have zero timestamps), so we can ignore the bounds' MVCC timestamps. - iter.SetUpperBound(end) - iter.SeekGE(MakeMVCCMetadataKey(start)) - - for ; ; iter.Next() { - valid, err := iter.Valid() - if err != nil { +// ClearIterRange implements the Batch interface. +func (p *pebbleBatch) ClearIterRange(start, end roachpb.Key) error { + iter := p.NewMVCCIterator(MVCCKeyAndIntentsIterKind, IterOptions{ + LowerBound: start, + UpperBound: end, + }) + defer iter.Close() + + for iter.SeekGE(MVCCKey{Key: start}); ; iter.Next() { + if valid, err := iter.Valid(); err != nil { return err } else if !valid { break } + // NB: UnsafeRawKey could be a serialized lock table key, and not just an // MVCCKey. - err = p.batch.Delete(iter.UnsafeRawKey(), nil) - if err != nil { + if err := p.batch.Delete(iter.UnsafeRawKey(), nil); err != nil { return err } } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 8fc315668657..c55a12f0efd5 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -39,10 +39,9 @@ type pebbleIterator struct { // use two slices for each of the bounds since this caller should not change // the slice holding the current bounds, that the callee (pebble.MVCCIterator) // is currently using, until after the caller has made the SetBounds call. - lowerBoundBuf [2][]byte - upperBoundBuf [2][]byte - curBuf int - testingSetBoundsListener testingSetBoundsListener + lowerBoundBuf [2][]byte + upperBoundBuf [2][]byte + curBuf int // Set to true to govern whether to call SeekPrefixGE or SeekGE. Skips // SSTables based on MVCC/Engine key when true. @@ -79,10 +78,6 @@ type cloneableIter interface { Close() error } -type testingSetBoundsListener interface { - postSetBounds(lower, upper []byte) -} - // Instantiates a new Pebble iterator, or gets one from the pool. func newPebbleIterator( handle pebble.Reader, @@ -265,9 +260,6 @@ func (p *pebbleIterator) setBounds(lowerBound, upperBound roachpb.Key) { p.options.UpperBound = p.upperBoundBuf[i] } p.iter.SetBounds(p.options.LowerBound, p.options.UpperBound) - if p.testingSetBoundsListener != nil { - p.testingSetBoundsListener.postSetBounds(p.options.LowerBound, p.options.UpperBound) - } } // Close implements the MVCCIterator interface. @@ -759,37 +751,6 @@ func findSplitKeyUsingIterator( return bestSplitKey, nil } -// SetUpperBound implements the MVCCIterator interface. Note that this is not -// the first time that bounds will be passed to the underlying -// pebble.Iterator. The existing bounds are in p.options. -func (p *pebbleIterator) SetUpperBound(upperBound roachpb.Key) { - if upperBound == nil { - panic("SetUpperBound must not use a nil key") - } - if p.options.UpperBound != nil { - // We know that we've appended 0x00 to p.options.UpperBound, which must be - // ignored for this comparison. - if bytes.Equal(p.options.UpperBound[:len(p.options.UpperBound)-1], upperBound) { - // Nothing to do. This noop optimization helps the underlying - // pebble.Iterator to optimize seeks. - return - } - } - p.curBuf = (p.curBuf + 1) % 2 - i := p.curBuf - if p.options.LowerBound != nil { - p.lowerBoundBuf[i] = append(p.lowerBoundBuf[i][:0], p.options.LowerBound...) - p.options.LowerBound = p.lowerBoundBuf[i] - } - p.upperBoundBuf[i] = append(p.upperBoundBuf[i][:0], upperBound...) - p.upperBoundBuf[i] = append(p.upperBoundBuf[i], 0x00) - p.options.UpperBound = p.upperBoundBuf[i] - p.iter.SetBounds(p.options.LowerBound, p.options.UpperBound) - if p.testingSetBoundsListener != nil { - p.testingSetBoundsListener.postSetBounds(p.options.LowerBound, p.options.UpperBound) - } -} - // Stats implements the {MVCCIterator,EngineIterator} interfaces. func (p *pebbleIterator) Stats() IteratorStats { return IteratorStats{ diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 44409107043d..19831b167079 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -228,141 +228,6 @@ func TestPebbleIterReuse(t *testing.T) { iter2.Close() } -type iterBoundsChecker struct { - t *testing.T - expectSetBounds bool - boundsSlices [2][]byte - boundsSlicesCopied [2][]byte -} - -func (ibc *iterBoundsChecker) postSetBounds(lower, upper []byte) { - require.True(ibc.t, ibc.expectSetBounds) - ibc.expectSetBounds = false - // The slices passed in the second from last SetBounds call - // must still be the same. - for i := range ibc.boundsSlices { - if ibc.boundsSlices[i] != nil { - if !bytes.Equal(ibc.boundsSlices[i], ibc.boundsSlicesCopied[i]) { - ibc.t.Fatalf("bound slice changed: expected: %x, actual: %x", - ibc.boundsSlicesCopied[i], ibc.boundsSlices[i]) - } - } - } - // Stash the bounds for later checking. - for i, bound := range [][]byte{lower, upper} { - ibc.boundsSlices[i] = bound - if bound != nil { - ibc.boundsSlicesCopied[i] = append(ibc.boundsSlicesCopied[i][:0], bound...) - } else { - ibc.boundsSlicesCopied[i] = nil - } - } -} - -func TestPebbleIterBoundSliceStabilityAndNoop(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - eng := createTestPebbleEngine().(*Pebble) - defer eng.Close() - iter := newPebbleIterator( - eng.db, nil, IterOptions{UpperBound: roachpb.Key("foo")}, StandardDurability) - defer iter.Close() - checker := &iterBoundsChecker{t: t} - iter.testingSetBoundsListener = checker - - tc := []struct { - expectSetBounds bool - setUpperOnly bool - lb roachpb.Key - ub roachpb.Key - }{ - { - // [nil, www) - expectSetBounds: true, - ub: roachpb.Key("www"), - }, - { - // [nil, www) - expectSetBounds: false, - ub: roachpb.Key("www"), - }, - { - // [nil, www) - expectSetBounds: false, - setUpperOnly: true, - ub: roachpb.Key("www"), - }, - { - // [ddd, www) - expectSetBounds: true, - lb: roachpb.Key("ddd"), - ub: roachpb.Key("www"), - }, - { - // [ddd, www) - expectSetBounds: false, - setUpperOnly: true, - ub: roachpb.Key("www"), - }, - { - // [ddd, xxx) - expectSetBounds: true, - setUpperOnly: true, - ub: roachpb.Key("xxx"), - }, - { - // [aaa, bbb) - expectSetBounds: true, - lb: roachpb.Key("aaa"), - ub: roachpb.Key("bbb"), - }, - { - // [ccc, ddd) - expectSetBounds: true, - lb: roachpb.Key("ccc"), - ub: roachpb.Key("ddd"), - }, - { - // [ccc, nil) - expectSetBounds: true, - lb: roachpb.Key("ccc"), - }, - { - // [ccc, nil) - expectSetBounds: false, - lb: roachpb.Key("ccc"), - }, - } - var lb, ub roachpb.Key - for _, c := range tc { - t.Run(fmt.Sprintf("%v", c), func(t *testing.T) { - checker.expectSetBounds = c.expectSetBounds - checker.t = t - if c.setUpperOnly { - iter.SetUpperBound(c.ub) - ub = c.ub - } else { - iter.setBounds(c.lb, c.ub) - lb, ub = c.lb, c.ub - } - require.False(t, checker.expectSetBounds) - for i, bound := range [][]byte{lb, ub} { - if (bound == nil) != (checker.boundsSlicesCopied[i] == nil) { - t.Fatalf("inconsistent nil %d", i) - } - if bound != nil { - expected := append([]byte(nil), bound...) - expected = append(expected, 0x00) - if !bytes.Equal(expected, checker.boundsSlicesCopied[i]) { - t.Fatalf("expected: %x, actual: %x", expected, checker.boundsSlicesCopied[i]) - } - } - } - }) - } -} - func makeMVCCKey(a string) MVCCKey { return MVCCKey{Key: []byte(a)} } diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 779767b26c1a..38067b313dad 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -285,7 +285,7 @@ func (fw *SSTWriter) SingleClearEngineKey(key EngineKey) error { } // ClearIterRange implements the Writer interface. -func (fw *SSTWriter) ClearIterRange(iter MVCCIterator, start, end roachpb.Key) error { +func (fw *SSTWriter) ClearIterRange(start, end roachpb.Key) error { panic("ClearIterRange is unsupported") } diff --git a/pkg/storage/testdata/intent_interleaving_iter/basic b/pkg/storage/testdata/intent_interleaving_iter/basic index 6f1f3c44f47e..d2af2a82f6d7 100644 --- a/pkg/storage/testdata/intent_interleaving_iter/basic +++ b/pkg/storage/testdata/intent_interleaving_iter/basic @@ -40,7 +40,6 @@ prev prev prev stats -set-upper k=c seek-ge k=b stats next @@ -75,16 +74,15 @@ prev: output: value k=a ts=20.000000000,0 v=a20 prev: output: meta k=a ts=20.000000000,0 txn=1 prev: output: . stats: (interface (dir, seek, step): (fwd, 2, 7), (rev, 0, 13)), (internal (dir, seek, step): (fwd, 2, 7), (rev, 2, 7)) -set-upper c seek-ge "b"/0,0: output: meta k=b ts=30.000000000,0 txn=2 stats: (interface (dir, seek, step): (fwd, 4, 7), (rev, 0, 13)), (internal (dir, seek, step): (fwd, 4, 7), (rev, 2, 7)) next: output: value k=b ts=30.000000000,0 v=b30 -next: output: . +next: output: meta k=c prev: output: value k=b ts=30.000000000,0 v=b30 prev: output: meta k=b ts=30.000000000,0 txn=2 prev: output: value k=a ts=10.000000000,0 v=a10 seek-lt "b"/0,0: output: value k=a ts=10.000000000,0 v=a10 -stats: (interface (dir, seek, step): (fwd, 4, 9), (rev, 2, 19)), (internal (dir, seek, step): (fwd, 4, 9), (rev, 6, 13)) +stats: (interface (dir, seek, step): (fwd, 4, 9), (rev, 2, 20)), (internal (dir, seek, step): (fwd, 4, 9), (rev, 5, 14)) next: output: meta k=b ts=30.000000000,0 txn=2 prev: output: value k=a ts=10.000000000,0 v=a10 prev: output: value k=a ts=20.000000000,0 v=a20