Skip to content

Commit

Permalink
storage: skip extra encoding in pebbleMVCCScanner
Browse files Browse the repository at this point in the history
Previously, we were re-encoding decoded keys in `pebbleMVCCScanner`.

This unnecessary encoding showed up on CPU profiles.

This change avoids the extra encoding by using `rawKey` instead of
MVCCKey.

Fixes #49145

Release note: None
  • Loading branch information
aadityasondhi committed Sep 30, 2020
1 parent 991893b commit 0236638
Show file tree
Hide file tree
Showing 7 changed files with 53 additions and 32 deletions.
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/spanset/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,11 @@ func (i *Iterator) UnsafeKey() storage.MVCCKey {
return i.i.UnsafeKey()
}

// UnsafeRawKey is part of the engine.Iterator interface.
func (i *Iterator) UnsafeRawKey() []byte {
return i.i.UnsafeRawKey()
}

// UnsafeValue is part of the engine.Iterator interface.
func (i *Iterator) UnsafeValue() []byte {
return i.i.UnsafeValue()
Expand Down
2 changes: 2 additions & 0 deletions pkg/storage/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,8 @@ type Iterator interface {
Prev()
// Key returns the current key.
Key() MVCCKey
// UnsafeRawKey returns the current raw key (i.e. the encoded MVCC key).
UnsafeRawKey() []byte
// Value returns the current value as a byte slice.
Value() []byte
// ValueProto unmarshals the value the iterator is currently
Expand Down
3 changes: 1 addition & 2 deletions pkg/storage/pebble_batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -270,7 +270,6 @@ func (p *pebbleBatch) ClearIterRange(iter Iterator, start, end roachpb.Key) erro
panic("distinct batch open")
}

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 with no
// lower bounds, calling SetUpperBound should be sufficient and safe.
Expand All @@ -287,7 +286,7 @@ func (p *pebbleBatch) ClearIterRange(iter Iterator, start, end roachpb.Key) erro
break
}

err = p.batch.Delete(iter.(unsafeRawKeyGetter).unsafeRawKey(), nil)
err = p.batch.Delete(iter.UnsafeRawKey(), nil)
if err != nil {
return err
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/pebble_iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -237,8 +237,8 @@ func (p *pebbleIterator) UnsafeKey() MVCCKey {
return mvccKey
}

// unsafeRawKey returns the raw key from the underlying pebble.Iterator.
func (p *pebbleIterator) unsafeRawKey() []byte {
// UnsafeRawKey returns the raw key from the underlying pebble.Iterator.
func (p *pebbleIterator) UnsafeRawKey() []byte {
return p.iter.Key()
}

Expand Down
55 changes: 32 additions & 23 deletions pkg/storage/pebble_mvcc_scanner.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ func (p *pebbleResults) clear() {
// The repr that MVCCScan / MVCCGet expects to provide as output goes:
// <valueLen:Uint32><keyLen:Uint32><Key><Value>
// This function adds to repr in that format.
func (p *pebbleResults) put(key MVCCKey, value []byte) {
func (p *pebbleResults) put(key []byte, value []byte) {
// Key value lengths take up 8 bytes (2 x Uint32).
const kvLenSize = 8
const minSize = 16
Expand All @@ -53,7 +53,7 @@ func (p *pebbleResults) put(key MVCCKey, value []byte) {
// We maintain a list of buffers, always encoding into the last one (a.k.a.
// pebbleResults.repr). The size of the buffers is exponentially increasing,
// capped at maxSize.
lenKey := key.Len()
lenKey := len(key)
lenToAdd := kvLenSize + lenKey + len(value)
if len(p.repr)+lenToAdd > cap(p.repr) {
newSize := 2 * cap(p.repr)
Expand All @@ -73,7 +73,7 @@ func (p *pebbleResults) put(key MVCCKey, value []byte) {
p.repr = p.repr[:startIdx+lenToAdd]
binary.LittleEndian.PutUint32(p.repr[startIdx:], uint32(len(value)))
binary.LittleEndian.PutUint32(p.repr[startIdx+4:], uint32(lenKey))
encodeKeyToBuf(p.repr[startIdx+kvLenSize:startIdx+kvLenSize+lenKey], key, lenKey)
copy(p.repr[startIdx+kvLenSize:], key)
copy(p.repr[startIdx+kvLenSize+lenKey:], value)
p.count++
p.bytes += int64(lenToAdd)
Expand Down Expand Up @@ -122,10 +122,11 @@ type pebbleMVCCScanner struct {
// cur* variables store the "current" record we're pointing to. Updated in
// updateCurrent. Note that the timestamp can be clobbered in the case of
// adding an intent from the intent history but is otherwise meaningful.
curKey MVCCKey
curValue []byte
results pebbleResults
intents pebble.Batch
curKey MVCCKey
curRawKey []byte
curValue []byte
results pebbleResults
intents pebble.Batch
// mostRecentTS stores the largest timestamp observed that is above the scan
// timestamp. Only applicable if failOnMoreRecent is true. If set and no
// other error is hit, a WriteToOld error will be returned from the scan.
Expand Down Expand Up @@ -287,7 +288,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool {
if p.curKey.Timestamp.LessEq(p.ts) {
// 1. Fast path: there is no intent and our read timestamp is newer than
// the most recent version's timestamp.
return p.addAndAdvance(p.curValue)
return p.addAndAdvance(p.curRawKey, p.curValue)
}

if p.failOnMoreRecent {
Expand Down Expand Up @@ -331,7 +332,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool {
}
if len(p.meta.RawBytes) != 0 {
// 5. Emit immediately if the value is inline.
return p.addAndAdvance(p.meta.RawBytes)
return p.addAndAdvance(p.curRawKey, p.meta.RawBytes)
}

if p.meta.Txn == nil {
Expand Down Expand Up @@ -381,8 +382,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool {
// that lie before the resume key.
return false
}
p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], p.curKey)
p.err = p.intents.Set(p.keyBuf, p.curValue, nil)
p.err = p.intents.Set(p.curRawKey, p.curValue, nil)
if p.err != nil {
return false
}
Expand All @@ -402,8 +402,7 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool {
// Note that this will trigger an error higher up the stack. We
// continue scanning so that we can return all of the intents
// in the scan range.
p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], p.curKey)
p.err = p.intents.Set(p.keyBuf, p.curValue, nil)
p.err = p.intents.Set(p.curRawKey, p.curValue, nil)
if p.err != nil {
return false
}
Expand Down Expand Up @@ -437,7 +436,8 @@ func (p *pebbleMVCCScanner) getAndAdvance() bool {
// about to advance. If this proves to be a problem later, we can extend
// addAndAdvance to take an MVCCKey explicitly.
p.curKey.Timestamp = metaTS
return p.addAndAdvance(value)
p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], p.curKey)
return p.addAndAdvance(p.keyBuf, value)
}
// 11. If no value in the intent history has a sequence number equal to
// or less than the read, we must ignore the intents laid down by the
Expand Down Expand Up @@ -576,14 +576,14 @@ func (p *pebbleMVCCScanner) advanceKeyAtNewKey(key []byte) bool {
return true
}

// Adds the specified value to the result set, excluding tombstones unless
// Adds the specified key and value to the result set, excluding tombstones unless
// p.tombstones is true. Advances to the next key unless we've reached the max
// results limit.
func (p *pebbleMVCCScanner) addAndAdvance(val []byte) bool {
func (p *pebbleMVCCScanner) addAndAdvance(rawKey []byte, val []byte) bool {
// Don't include deleted versions len(val) == 0, unless we've been instructed
// to include tombstones in the results.
if len(val) > 0 || p.tombstones {
p.results.put(p.curKey, val)
p.results.put(rawKey, val)
if p.targetBytes > 0 && p.results.bytes >= p.targetBytes {
// When the target bytes are met or exceeded, stop producing more
// keys. We implement this by reducing maxKeys to the current
Expand Down Expand Up @@ -620,7 +620,7 @@ func (p *pebbleMVCCScanner) seekVersion(ts hlc.Timestamp, uncertaintyCheck bool)
if uncertaintyCheck && p.ts.Less(p.curKey.Timestamp) {
return p.uncertaintyError(p.curKey.Timestamp)
}
return p.addAndAdvance(p.curValue)
return p.addAndAdvance(p.curRawKey, p.curValue)
}
}

Expand All @@ -635,7 +635,7 @@ func (p *pebbleMVCCScanner) seekVersion(ts hlc.Timestamp, uncertaintyCheck bool)
if uncertaintyCheck && p.ts.Less(p.curKey.Timestamp) {
return p.uncertaintyError(p.curKey.Timestamp)
}
return p.addAndAdvance(p.curValue)
return p.addAndAdvance(p.curRawKey, p.curValue)
}
return p.advanceKey()
}
Expand All @@ -646,7 +646,13 @@ func (p *pebbleMVCCScanner) updateCurrent() bool {
return false
}

p.curKey = p.parent.UnsafeKey()
p.curRawKey = p.parent.UnsafeRawKey()

var err error
p.curKey, err = DecodeMVCCKey(p.curRawKey)
if err != nil {
panic(err)
}
p.curValue = p.parent.UnsafeValue()
return true
}
Expand Down Expand Up @@ -728,10 +734,13 @@ func (p *pebbleMVCCScanner) iterPeekPrev() ([]byte, bool) {
// We need to save a copy of the current iterator key and value and adjust
// curRawKey, curKey and curValue to point to this saved data. We use a
// single buffer for this purpose: savedBuf.
p.savedBuf = append(p.savedBuf[:0], p.curKey.Key...)
p.savedBuf = append(p.savedBuf[:0], p.curRawKey...)
p.savedBuf = append(p.savedBuf, p.curValue...)
p.curKey.Key = p.savedBuf[:len(p.curKey.Key)]
p.curValue = p.savedBuf[len(p.curKey.Key):]
p.curRawKey = p.savedBuf[:len(p.curRawKey)]
p.curValue = p.savedBuf[len(p.curRawKey):]
// The raw key is always a prefix of the encoded MVCC key. Take advantage of this to
// sub-slice the raw key directly, instead of calling SplitMVCCKey.
p.curKey.Key = p.curRawKey[:len(p.curKey.Key)]

// With the current iterator state saved we can move the iterator to the
// previous entry.
Expand Down
8 changes: 8 additions & 0 deletions pkg/storage/rocksdb.go
Original file line number Diff line number Diff line change
Expand Up @@ -1299,6 +1299,10 @@ func (r *batchIterator) UnsafeKey() MVCCKey {
return r.iter.UnsafeKey()
}

func (r *batchIterator) UnsafeRawKey() []byte {
return r.iter.UnsafeRawKey()
}

func (r *batchIterator) UnsafeValue() []byte {
return r.iter.UnsafeValue()
}
Expand Down Expand Up @@ -2027,6 +2031,10 @@ func (r *rocksDBIterator) UnsafeKey() MVCCKey {
return cToUnsafeGoKey(r.key)
}

func (r *rocksDBIterator) UnsafeRawKey() []byte {
panic("unimplemented; only meant to be used with Pebble")
}

func (r *rocksDBIterator) UnsafeValue() []byte {
return cSliceToUnsafeGoBytes(r.value)
}
Expand Down
8 changes: 3 additions & 5 deletions pkg/storage/tee.go
Original file line number Diff line number Diff line change
Expand Up @@ -1166,11 +1166,9 @@ func (t *TeeEngineIter) Key() MVCCKey {
return t.iter1.Key()
}

func (t *TeeEngineIter) unsafeRawKey() []byte {
type unsafeRawKeyGetter interface {
unsafeRawKey() []byte
}
return t.iter1.(unsafeRawKeyGetter).unsafeRawKey()
// UnsafeRawKey implements the Iterator interface.
func (t *TeeEngineIter) UnsafeRawKey() []byte {
return t.iter1.UnsafeRawKey()
}

// Value implements the Iterator interface.
Expand Down

0 comments on commit 0236638

Please sign in to comment.