From 25ab18038bb2aa030094fe1d282b32a24ffff35c Mon Sep 17 00:00:00 2001 From: Bilal Akhtar Date: Fri, 11 Oct 2019 16:34:46 -0400 Subject: [PATCH] engine: Fix case where a pebbleBatchIterator is passed to ClearIterRange Currently ClearIterRange expects a pebbleIterator to be passed in as the iterator. This isn't always the case; occasionally, we see a pebbleBatchIterator which currently errors out the method. Update that method in pebble and pebbleBatch to accept both types of iterators. Release note: None --- pkg/storage/engine/engine.go | 4 ++++ pkg/storage/engine/pebble.go | 33 +++++---------------------- pkg/storage/engine/pebble_batch.go | 22 +++++++----------- pkg/storage/engine/pebble_iterator.go | 5 ++++ 4 files changed, 23 insertions(+), 41 deletions(-) diff --git a/pkg/storage/engine/engine.go b/pkg/storage/engine/engine.go index a21bbdc42cda..d49cf5a18a8e 100644 --- a/pkg/storage/engine/engine.go +++ b/pkg/storage/engine/engine.go @@ -241,6 +241,10 @@ type Writer interface { // // It is safe to modify the contents of the arguments after ClearIterRange // returns. + // + // TODO(itsbilal): All calls to ClearIterRange pass in metadata keys for + // start and end that have a zero timestamp. Change the type of those args + // to roachpb.Key to make this expectation explicit. ClearIterRange(iter Iterator, start, end MVCCKey) 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/pebble.go b/pkg/storage/engine/pebble.go index 718072cd883f..24e99c359e87 100644 --- a/pkg/storage/engine/pebble.go +++ b/pkg/storage/engine/pebble.go @@ -278,35 +278,14 @@ func (p *Pebble) ClearIterRange(iter Iterator, start, end MVCCKey) error { panic("write operation called on read-only pebble instance") } - pebbleIter, ok := iter.(*pebbleIterator) - if !ok { - return errors.Errorf("%T is not a pebble iterator", iter) - } - // Note that this method has the side effect of modifying iter's bounds. - // Since all calls to `ClearIterRange` are on new throwaway iterators, this - // should be fine. - pebbleIter.lowerBoundBuf = EncodeKeyToBuf(pebbleIter.lowerBoundBuf[:0], start) - pebbleIter.options.LowerBound = pebbleIter.lowerBoundBuf - pebbleIter.upperBoundBuf = EncodeKeyToBuf(pebbleIter.upperBoundBuf[:0], end) - pebbleIter.options.UpperBound = pebbleIter.upperBoundBuf - pebbleIter.iter.SetBounds(pebbleIter.lowerBoundBuf, pebbleIter.upperBoundBuf) - - pebbleIter.Seek(start) - for ; ; pebbleIter.Next() { - ok, err := pebbleIter.Valid() - if err != nil { - return err - } else if !ok || !pebbleIter.UnsafeKey().Less(end) { - break - } + // Write all the tombstones in one batch. + batch := p.NewWriteOnlyBatch() + defer batch.Close() - err = p.db.Delete(pebbleIter.iter.Key(), pebble.Sync) - if err != nil { - return err - } + if err := batch.ClearIterRange(iter, start, end); err != nil { + return err } - - return nil + return batch.Commit(true) } // Merge implements the Engine interface. diff --git a/pkg/storage/engine/pebble_batch.go b/pkg/storage/engine/pebble_batch.go index cf1e8f16a471..cc23ce693ed5 100644 --- a/pkg/storage/engine/pebble_batch.go +++ b/pkg/storage/engine/pebble_batch.go @@ -14,7 +14,6 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/util/protoutil" - "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble" ) @@ -197,18 +196,14 @@ func (p *pebbleBatch) ClearIterRange(iter Iterator, start, end MVCCKey) error { panic("distinct batch open") } - pebbleIter, ok := iter.(*pebbleIterator) - if !ok { - return errors.Errorf("%T is not a pebble iterator", iter) - } + type unsafeRawKeyGetter interface{ unsafeRawKey() []byte } // Note that this method has the side effect of modifying iter's bounds. - // Since all calls to `ClearIterRange` are on new throwaway iterators, this - // should be fine. - pebbleIter.lowerBoundBuf = EncodeKeyToBuf(pebbleIter.lowerBoundBuf[:0], start) - pebbleIter.options.LowerBound = pebbleIter.lowerBoundBuf - pebbleIter.upperBoundBuf = EncodeKeyToBuf(pebbleIter.upperBoundBuf[:0], end) - pebbleIter.options.UpperBound = pebbleIter.upperBoundBuf - pebbleIter.iter.SetBounds(pebbleIter.lowerBoundBuf, pebbleIter.upperBoundBuf) + // 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.Key) + iter.Seek(start) for ; ; iter.Next() { valid, err := iter.Valid() @@ -218,8 +213,7 @@ func (p *pebbleBatch) ClearIterRange(iter Iterator, start, end MVCCKey) error { break } - p.buf = EncodeKeyToBuf(p.buf[:0], iter.Key()) - err = p.batch.Delete(p.buf, nil) + err = p.batch.Delete(iter.(unsafeRawKeyGetter).unsafeRawKey(), nil) if err != nil { return err } diff --git a/pkg/storage/engine/pebble_iterator.go b/pkg/storage/engine/pebble_iterator.go index 7df1e3e6f7a5..96424f218a94 100644 --- a/pkg/storage/engine/pebble_iterator.go +++ b/pkg/storage/engine/pebble_iterator.go @@ -159,6 +159,11 @@ func (p *pebbleIterator) UnsafeKey() MVCCKey { return mvccKey } +// unsafeRawKey returns the raw key from the underlying pebble.Iterator. +func (p *pebbleIterator) unsafeRawKey() []byte { + return p.iter.Key() +} + // UnsafeValue implements the Iterator interface. func (p *pebbleIterator) UnsafeValue() []byte { if valid, err := p.Valid(); err != nil || !valid {