From 2e5bae5e88383270bad3babf26404198815a4052 Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Thu, 11 Feb 2021 13:39:01 -0500 Subject: [PATCH] [DNM] storage: add checking around reuse of pebbleMVCCScanner.keyBuf The logic is hard to reason about. There are call paths like seekVersion -> advanceKeyAtNewKey -> prevKey that overwrite what is in the keyBuf (though probably harmless since it is overwriting keyBuf with itself). Release note: None --- pkg/storage/mvcc.go | 2 ++ pkg/storage/pebble_mvcc_scanner.go | 55 ++++++++++++++++++++++++++---- 2 files changed, 50 insertions(+), 7 deletions(-) diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index b00822e531db..884c9d0b77d7 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -880,6 +880,7 @@ func mvccGet( tombstones: opts.Tombstones, failOnMoreRecent: opts.FailOnMoreRecent, keyBuf: mvccScanner.keyBuf, + // savedBuf: mvccScanner.savedBuf, } mvccScanner.init(opts.Txn, opts.LocalUncertaintyLimit) @@ -2336,6 +2337,7 @@ func mvccScanToBytes( tombstones: opts.Tombstones, failOnMoreRecent: opts.FailOnMoreRecent, keyBuf: mvccScanner.keyBuf, + // savedBuf: mvccScanner.savedBuf, } mvccScanner.init(opts.Txn, opts.LocalUncertaintyLimit) diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 342019cca366..3324042c711a 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -122,6 +122,7 @@ type pebbleMVCCScanner struct { failOnMoreRecent bool isGet bool keyBuf []byte + keyBufInUse bool savedBuf []byte // cur* variables store the "current" record we're pointing to. Updated in // updateCurrent. Note that the timestamp can be clobbered in the case of @@ -516,14 +517,20 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool { // nextKey advances to the next user key. func (p *pebbleMVCCScanner) nextKey() bool { + if p.keyBufInUse { + panic("keyBufInUse in nextKey") + } + p.keyBufInUse = true p.keyBuf = append(p.keyBuf[:0], p.curUnsafeKey.Key...) for i := 0; i < p.itersBeforeSeek; i++ { if !p.iterNext() { + p.keyBufInUse = false return false } if !bytes.Equal(p.curUnsafeKey.Key, p.keyBuf) { p.incrementItersBeforeSeek() + p.keyBufInUse = false return true } } @@ -536,6 +543,7 @@ func (p *pebbleMVCCScanner) nextKey() bool { // MVCCIterator is free to mutate the backing for p.curUnsafeKey.Key // arbitrarily. Therefore we use p.keyBuf here which we have handy. p.keyBuf = append(p.keyBuf, 0) + p.keyBufInUse = false return p.iterSeek(MVCCKey{Key: p.keyBuf}) } @@ -545,19 +553,26 @@ func (p *pebbleMVCCScanner) nextKey() bool { // iterator was exhausted. Assumes that the iterator is currently positioned at // the oldest version of key. func (p *pebbleMVCCScanner) backwardLatestVersion(key []byte, i int) bool { + if p.keyBufInUse { + panic("keyBufInUse in backwardLatestVersion") + } + p.keyBufInUse = true p.keyBuf = append(p.keyBuf[:0], key...) for ; i < p.itersBeforeSeek; i++ { peekedKey, ok := p.iterPeekPrev() if !ok { + p.keyBufInUse = false // No previous entry exists, so we're at the latest version of key. return true } if !bytes.Equal(peekedKey, p.keyBuf) { p.incrementItersBeforeSeek() + p.keyBufInUse = false return true } if !p.iterPrev() { + p.keyBufInUse = false return false } } @@ -568,30 +583,41 @@ func (p *pebbleMVCCScanner) backwardLatestVersion(key []byte, i int) bool { // the underlying MVCCIterator is free to mutate the backing for key // arbitrarily. Therefore we use p.keyBuf here which we have handy. p.decrementItersBeforeSeek() + p.keyBufInUse = false return p.iterSeek(MVCCKey{Key: p.keyBuf}) } // prevKey advances to the newest version of the user key preceding the // specified key. Assumes that the iterator is currently positioned at // key or 1 record after key. -func (p *pebbleMVCCScanner) prevKey(key []byte) bool { - p.keyBuf = append(p.keyBuf[:0], key...) - +func (p *pebbleMVCCScanner) prevKey(key []byte, unsafeKey bool) bool { + if unsafeKey { + if p.keyBufInUse { + panic("keyBufInUse in prevKey") + } + p.keyBufInUse = true + p.keyBuf = append(p.keyBuf[:0], key...) + key = p.keyBuf + } for i := 0; i < p.itersBeforeSeek; i++ { peekedKey, ok := p.iterPeekPrev() if !ok { + p.keyBufInUse = false return false } - if !bytes.Equal(peekedKey, p.keyBuf) { + if !bytes.Equal(peekedKey, key) { + p.keyBufInUse = false return p.backwardLatestVersion(peekedKey, i+1) } if !p.iterPrev() { + p.keyBufInUse = false return false } } p.decrementItersBeforeSeek() - return p.iterSeekReverse(MVCCKey{Key: p.keyBuf}) + p.keyBufInUse = false + return p.iterSeekReverse(MVCCKey{Key: key}) } // advanceKey advances to the next key in the iterator's direction. @@ -600,7 +626,7 @@ func (p *pebbleMVCCScanner) advanceKey() bool { return false } if p.reverse { - return p.prevKey(p.curUnsafeKey.Key) + return p.prevKey(p.curUnsafeKey.Key, true /* unsafeKey */) } return p.nextKey() } @@ -624,12 +650,16 @@ func (p *pebbleMVCCScanner) advanceKeyAtEnd() bool { // advanceKeyAtNewKey advances to the key after the specified key, assuming we // have just reached the specified key. +// REQUIRES: key is backed by p.keyBuf and caller does not care if p.keyBuf +// is overwritten. +// It will set p.keyBufInUse=false before returning. func (p *pebbleMVCCScanner) advanceKeyAtNewKey(key []byte) bool { if p.reverse { // We've advanced to the next key but need to move back to the previous // key. - return p.prevKey(key) + return p.prevKey(key, false /* unsafeKey */) } + p.keyBufInUse = false // We're already at the new key so there is nothing to do. return true } @@ -662,6 +692,10 @@ func (p *pebbleMVCCScanner) addAndAdvance(rawKey []byte, val []byte) bool { // the next user key. func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck bool) bool { seekKey := MVCCKey{Key: p.curUnsafeKey.Key, Timestamp: seekTS} + if p.keyBufInUse { + panic("keyBufInUse in seekVersion") + } + p.keyBufInUse = true p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], seekKey) origKey := p.keyBuf[:len(p.curUnsafeKey.Key)] // We will need seekKey below, if the next's don't suffice. Even though the @@ -672,6 +706,7 @@ func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck b for i := 0; i < p.itersBeforeSeek; i++ { if !p.iterNext() { + p.keyBufInUse = false return p.advanceKeyAtEnd() } if !bytes.Equal(p.curUnsafeKey.Key, origKey) { @@ -681,6 +716,7 @@ func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck b if p.curUnsafeKey.Timestamp.LessEq(seekTS) { p.incrementItersBeforeSeek() if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { + p.keyBufInUse = false return p.addAndAdvance(p.curRawKey, p.curValue) } // Iterate through uncertainty interval. Though we found a value in @@ -692,6 +728,7 @@ func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck b // any value with a time in the range (ts, localUncertaintyLimit] // is uncertain. if p.isUncertainValue(p.curUnsafeKey.Timestamp) { + p.keyBufInUse = false return p.uncertaintyError(p.curUnsafeKey.Timestamp) } } @@ -699,6 +736,7 @@ func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck b p.decrementItersBeforeSeek() if !p.iterSeek(seekKey) { + p.keyBufInUse = false return p.advanceKeyAtEnd() } for { @@ -706,15 +744,18 @@ func (p *pebbleMVCCScanner) seekVersion(seekTS hlc.Timestamp, uncertaintyCheck b return p.advanceKeyAtNewKey(origKey) } if !uncertaintyCheck || p.curUnsafeKey.Timestamp.LessEq(p.ts) { + p.keyBufInUse = false return p.addAndAdvance(p.curRawKey, p.curValue) } // Iterate through uncertainty interval. See the comment above about why // a value in this interval is not necessarily cause for an uncertainty // error. if p.isUncertainValue(p.curUnsafeKey.Timestamp) { + p.keyBufInUse = false return p.uncertaintyError(p.curUnsafeKey.Timestamp) } if !p.iterNext() { + p.keyBufInUse = false return p.advanceKeyAtEnd() } }