Skip to content

Commit

Permalink
engine: Fix case where a pebbleBatchIterator is passed to ClearIterRange
Browse files Browse the repository at this point in the history
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
  • Loading branch information
itsbilal committed Oct 14, 2019
1 parent 4b1974c commit 25ab180
Show file tree
Hide file tree
Showing 4 changed files with 23 additions and 41 deletions.
4 changes: 4 additions & 0 deletions pkg/storage/engine/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
33 changes: 6 additions & 27 deletions pkg/storage/engine/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
22 changes: 8 additions & 14 deletions pkg/storage/engine/pebble_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"sync"

"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
)

Expand Down Expand Up @@ -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()
Expand All @@ -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
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/storage/engine/pebble_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down

0 comments on commit 25ab180

Please sign in to comment.