diff --git a/pkg/kv/kvserver/spanset/batch.go b/pkg/kv/kvserver/spanset/batch.go index 3056cd74efbb..0b65860f4f3e 100644 --- a/pkg/kv/kvserver/spanset/batch.go +++ b/pkg/kv/kvserver/spanset/batch.go @@ -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() diff --git a/pkg/storage/engine.go b/pkg/storage/engine.go index 3f6ac3075f37..c361e300c6e6 100644 --- a/pkg/storage/engine.go +++ b/pkg/storage/engine.go @@ -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 diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index 5c6185676393..63e1de2621d2 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -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. @@ -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 } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index d5f7c3a57f42..1b8206685f6b 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -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() } diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index f07e92abd281..3f717de80ab9 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -44,7 +44,7 @@ func (p *pebbleResults) clear() { // The repr that MVCCScan / MVCCGet expects to provide as output goes: // // 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 @@ -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) @@ -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) @@ -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. @@ -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 { @@ -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 { @@ -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 } @@ -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 } @@ -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 @@ -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 @@ -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) } } @@ -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() } @@ -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 } @@ -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. diff --git a/pkg/storage/rocksdb.go b/pkg/storage/rocksdb.go index e5dc2fad480d..c88652a40a12 100644 --- a/pkg/storage/rocksdb.go +++ b/pkg/storage/rocksdb.go @@ -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() } @@ -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) } diff --git a/pkg/storage/tee.go b/pkg/storage/tee.go index c858bd06cf20..04f528a400f7 100644 --- a/pkg/storage/tee.go +++ b/pkg/storage/tee.go @@ -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.