Skip to content

Commit

Permalink
Merge #75721
Browse files Browse the repository at this point in the history
75721: storage: clean up MVCC key encoding functions r=jbowens,sumeerbhola a=erikgrinaker

**storage: move MVCC key/encoding logic into separate file**

This splits MVCC keys and their encoding into a separate file, following
the same convention as `engine_key.go`. There is additional decoding
logic in `enginepb/decode.go` which has not been moved -- this was
originally to avoid a RocksDB CGo dependency for these functions, but
Pebble still has a (tiny) CGo dependency so we leave them for now.

Release note: None

**storage: include MVCC in names of key encoding functions**

Release note: None

**storage: clean up MVCC key encoding functions**

In particular, this separates the timestamp encoding from the overall
key encoding. This is necessary for the MVCC range tombstone work, where
Pebble key suffixes (timestamps) are processed in isolation.

Unfortunately, this adds ~9% overhead for `EncodeMVCCKey()`. This was
found to be due to additional function call overhead, and the Go
compiler's unwillingness to inline these. This was considered acceptable
for the encode path, while with the hotter `DecodeMVCCKey()` path the
timestamp decoding logic was instead duplicated to avoid this overhead.

```
name                                            old time/op    new time/op    delta
EncodeMVCCKey/key=empty/ts=empty-24               15.8ns ± 0%    15.7ns ± 0%   -0.46%  (p=0.000 n=10+10)
EncodeMVCCKey/key=empty/ts=walltime-24            17.9ns ± 0%    19.5ns ± 0%   +8.88%  (p=0.000 n=10+10)
EncodeMVCCKey/key=empty/ts=walltime+logical-24    18.5ns ± 0%    20.1ns ± 0%   +8.99%  (p=0.000 n=10+10)
EncodeMVCCKey/key=empty/ts=all-24                 18.8ns ± 0%    20.4ns ± 0%   +8.66%  (p=0.000 n=10+10)
EncodeMVCCKey/key=short/ts=walltime+logical-24    19.1ns ± 0%    20.7ns ± 0%   +8.38%  (p=0.000 n=10+9)
EncodeMVCCKey/key=short/ts=all-24                 19.5ns ± 0%    20.7ns ± 0%   +6.18%  (p=0.000 n=10+9)
EncodeMVCCKey/key=short/ts=empty-24               16.3ns ± 0%    16.0ns ± 0%   -1.86%  (p=0.000 n=10+10)
EncodeMVCCKey/key=short/ts=walltime-24            18.1ns ± 0%    20.6ns ± 0%  +13.41%  (p=0.000 n=8+8)
EncodeMVCCKey/key=long/ts=empty-24                58.7ns ± 0%    58.8ns ± 0%   +0.15%  (p=0.000 n=10+10)
EncodeMVCCKey/key=long/ts=walltime-24             59.8ns ± 0%    60.8ns ± 0%   +1.78%  (p=0.000 n=10+9)
EncodeMVCCKey/key=long/ts=walltime+logical-24     60.7ns ± 0%    61.7ns ± 0%   +1.54%  (p=0.000 n=10+10)
EncodeMVCCKey/key=long/ts=all-24                  60.9ns ± 0%    61.9ns ± 0%   +1.60%  (p=0.000 n=10+9)
DecodeMVCCKey/key=empty/ts=empty-24               12.4ns ± 0%    12.4ns ± 0%     ~     (p=0.912 n=10+6)
DecodeMVCCKey/key=empty/ts=walltime-24            13.3ns ± 0%    13.3ns ± 0%     ~     (p=0.054 n=10+10)
DecodeMVCCKey/key=empty/ts=walltime+logical-24    13.3ns ± 0%    13.3ns ± 0%   -0.06%  (p=0.034 n=10+10)
DecodeMVCCKey/key=empty/ts=all-24                 13.6ns ± 0%    13.6ns ± 0%     ~     (p=0.509 n=10+10)
DecodeMVCCKey/key=short/ts=walltime+logical-24    13.3ns ± 0%    13.3ns ± 0%     ~     (all equal)
DecodeMVCCKey/key=short/ts=all-24                 13.6ns ± 0%    13.6ns ± 0%     ~     (p=0.151 n=10+10)
DecodeMVCCKey/key=short/ts=empty-24               12.5ns ± 0%    12.4ns ± 0%   -0.21%  (p=0.000 n=10+10)
DecodeMVCCKey/key=short/ts=walltime-24            13.3ns ± 0%    13.3ns ± 0%     ~     (p=0.577 n=8+10)
DecodeMVCCKey/key=long/ts=walltime+logical-24     13.3ns ± 0%    13.3ns ± 0%     ~     (all equal)
DecodeMVCCKey/key=long/ts=all-24                  13.6ns ± 0%    13.6ns ± 0%     ~     (p=0.650 n=10+10)
DecodeMVCCKey/key=long/ts=empty-24                12.4ns ± 0%    12.4ns ± 0%   +0.15%  (p=0.004 n=10+10)
DecodeMVCCKey/key=long/ts=walltime-24             13.3ns ± 0%    13.3ns ± 0%   +0.10%  (p=0.012 n=10+9)
```

Release note: None

Co-authored-by: Erik Grinaker <grinaker@cockroachlabs.com>
  • Loading branch information
craig[bot] and erikgrinaker committed Feb 1, 2022
2 parents 19a5d53 + a72dbea commit c4f15d6
Show file tree
Hide file tree
Showing 25 changed files with 605 additions and 280 deletions.
4 changes: 2 additions & 2 deletions pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -642,8 +642,8 @@ func runDebugRaftLog(cmd *cobra.Command, args []string) error {
end := keys.RaftLogPrefix(rangeID).PrefixEnd()
fmt.Printf("Printing keys %s -> %s (RocksDB keys: %#x - %#x )\n",
start, end,
string(storage.EncodeKey(storage.MakeMVCCMetadataKey(start))),
string(storage.EncodeKey(storage.MakeMVCCMetadataKey(end))))
string(storage.EncodeMVCCKey(storage.MakeMVCCMetadataKey(start))),
string(storage.EncodeMVCCKey(storage.MakeMVCCMetadataKey(end))))

// NB: raft log does not have intents.
return db.MVCCIterate(start, end, storage.MVCCKeyIterKind, func(kv storage.MVCCKeyValue) error {
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/flags_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -155,7 +155,7 @@ func (k *mvccKey) Set(value string) error {
}
newK, err := storage.DecodeMVCCKey(b)
if err != nil {
encoded := gohex.EncodeToString(storage.EncodeKey(storage.MakeMVCCMetadataKey(roachpb.Key(b))))
encoded := gohex.EncodeToString(storage.EncodeMVCCKey(storage.MakeMVCCMetadataKey(roachpb.Key(b))))
return errors.Wrapf(err, "perhaps this is just a hex-encoded key; you need an "+
"encoded MVCCKey (i.e. with a timestamp component); here's one with a zero timestamp: %s",
encoded)
Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvnemesis/engine.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,7 @@ func (e *Engine) Close() {
func (e *Engine) Get(key roachpb.Key, ts hlc.Timestamp) roachpb.Value {
iter := e.kvs.NewIter(nil)
defer func() { _ = iter.Close() }()
iter.SeekGE(storage.EncodeKey(storage.MVCCKey{Key: key, Timestamp: ts}))
iter.SeekGE(storage.EncodeMVCCKey(storage.MVCCKey{Key: key, Timestamp: ts}))
if !iter.Valid() {
return roachpb.Value{}
}
Expand All @@ -78,15 +78,15 @@ func (e *Engine) Get(key roachpb.Key, ts hlc.Timestamp) roachpb.Value {
// Put inserts a key/value/timestamp tuple. If an exact key/timestamp pair is
// Put again, it overwrites the previous value.
func (e *Engine) Put(key storage.MVCCKey, value []byte) {
if err := e.kvs.Set(storage.EncodeKey(key), value, nil); err != nil {
if err := e.kvs.Set(storage.EncodeMVCCKey(key), value, nil); err != nil {
panic(err)
}
}

// Delete writes a tombstone value for a given key/timestamp. This is
// equivalent to a Put with an empty value.
func (e *Engine) Delete(key storage.MVCCKey) {
if err := e.kvs.Set(storage.EncodeKey(key), nil, nil); err != nil {
if err := e.kvs.Set(storage.EncodeMVCCKey(key), nil, nil); err != nil {
panic(err)
}
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvnemesis/validator.go
Original file line number Diff line number Diff line change
Expand Up @@ -658,7 +658,7 @@ func (v *validator) checkCommittedTxn(
}
if !o.Timestamp.IsEmpty() {
mvccKey := storage.MVCCKey{Key: o.Key, Timestamp: o.Timestamp}
if err := batch.Delete(storage.EncodeKey(mvccKey), nil); err != nil {
if err := batch.Delete(storage.EncodeMVCCKey(mvccKey), nil); err != nil {
panic(err)
}
}
Expand Down Expand Up @@ -693,7 +693,7 @@ func (v *validator) checkCommittedTxn(
Timestamp: txnObservations[lastWriteIdx].(*observedWrite).Timestamp,
}
}
if err := batch.Set(storage.EncodeKey(mvccKey), o.Value.RawBytes, nil); err != nil {
if err := batch.Set(storage.EncodeMVCCKey(mvccKey), o.Value.RawBytes, nil); err != nil {
panic(err)
}
case *observedRead:
Expand Down Expand Up @@ -931,7 +931,7 @@ func validReadTimes(

iter := b.NewIter(nil)
defer func() { _ = iter.Close() }()
iter.SeekGE(storage.EncodeKey(storage.MVCCKey{Key: key}))
iter.SeekGE(storage.EncodeMVCCKey(storage.MVCCKey{Key: key}))
for ; iter.Valid(); iter.Next() {
mvccKey, err := storage.DecodeMVCCKey(iter.Key())
if err != nil {
Expand Down Expand Up @@ -996,7 +996,7 @@ func validScanTime(
missingKeys := make(map[string]disjointTimeSpans)
iter := b.NewIter(nil)
defer func() { _ = iter.Close() }()
iter.SeekGE(storage.EncodeKey(storage.MVCCKey{Key: span.Key}))
iter.SeekGE(storage.EncodeMVCCKey(storage.MVCCKey{Key: span.Key}))
for ; iter.Valid(); iter.Next() {
mvccKey, err := storage.DecodeMVCCKey(iter.Key())
if err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/debug_print.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ func SprintEngineKey(key storage.EngineKey) string {

// SprintMVCCKey pretty-prints the specified MVCCKey.
func SprintMVCCKey(key storage.MVCCKey) string {
return fmt.Sprintf("%s %s (%#x): ", key.Timestamp, key.Key, storage.EncodeKey(key))
return fmt.Sprintf("%s %s (%#x): ", key.Timestamp, key.Key, storage.EncodeMVCCKey(key))
}

// SprintEngineKeyValue is like PrintEngineKeyValue, but returns a string. In
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/debug_print_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func TestStringifyWriteBatch(t *testing.T) {
}

batch := pebble.Batch{}
require.NoError(t, batch.Set(storage.EncodeKey(storage.MVCCKey{
require.NoError(t, batch.Set(storage.EncodeMVCCKey(storage.MVCCKey{
Key: roachpb.Key("/db1"),
Timestamp: hlc.Timestamp{WallTime: math.MaxInt64},
}), []byte("test value"), nil /* WriteOptions */))
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_consistency_diff.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func (rsds ReplicaSnapshotDiffSlice) SafeFormat(buf redact.SafePrinter, _ rune)
prefix, d.Timestamp, d.Key,
prefix, d.Timestamp.GoTime(),
prefix, SprintMVCCKeyValue(storage.MVCCKeyValue{Key: mvccKey, Value: d.Value}, false /* printKey */),
prefix, storage.EncodeKey(mvccKey), d.Value)
prefix, storage.EncodeMVCCKey(mvccKey), d.Value)
}
}

Expand Down
2 changes: 2 additions & 0 deletions pkg/storage/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ go_library(
"multi_iterator.go",
"mvcc.go",
"mvcc_incremental_iterator.go",
"mvcc_key.go",
"mvcc_logical_ops.go",
"open.go",
"pebble.go",
Expand Down Expand Up @@ -105,6 +106,7 @@ go_test(
"multi_iterator_test.go",
"mvcc_history_test.go",
"mvcc_incremental_iterator_test.go",
"mvcc_key_test.go",
"mvcc_logical_ops_test.go",
"mvcc_stats_test.go",
"mvcc_test.go",
Expand Down
62 changes: 0 additions & 62 deletions pkg/storage/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,8 +14,6 @@ import (
"encoding/binary"
"math"

"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
)
Expand Down Expand Up @@ -53,66 +51,6 @@ const (
countPos int = 8
)

// EncodeKey encodes an engine.MVCC key into the RocksDB representation.
func EncodeKey(key MVCCKey) []byte {
keyLen := key.Len()
buf := make([]byte, keyLen)
encodeKeyToBuf(buf, key, keyLen)
return buf
}

// EncodeKeyToBuf encodes an engine.MVCC key into the RocksDB representation.
func EncodeKeyToBuf(buf []byte, key MVCCKey) []byte {
keyLen := key.Len()
if cap(buf) < keyLen {
buf = make([]byte, keyLen)
} else {
buf = buf[:keyLen]
}
encodeKeyToBuf(buf, key, keyLen)
return buf
}

func encodeKeyToBuf(buf []byte, key MVCCKey, keyLen int) {
const (
timestampSentinelLen = 1
walltimeEncodedLen = 8
logicalEncodedLen = 4
syntheticEncodedLen = 1
)

copy(buf, key.Key)

pos := len(key.Key)
timestampLength := keyLen - pos - 1
if timestampLength > 0 {
buf[pos] = 0
pos += timestampSentinelLen
binary.BigEndian.PutUint64(buf[pos:], uint64(key.Timestamp.WallTime))
pos += walltimeEncodedLen
if key.Timestamp.Logical != 0 || key.Timestamp.Synthetic {
binary.BigEndian.PutUint32(buf[pos:], uint32(key.Timestamp.Logical))
pos += logicalEncodedLen
}
if key.Timestamp.Synthetic {
buf[pos] = 1
pos += syntheticEncodedLen
}
}
buf[len(buf)-1] = byte(timestampLength)
}

func encodeTimestamp(ts hlc.Timestamp) []byte {
_, encodedTS, _ := enginepb.SplitMVCCKey(EncodeKey(MVCCKey{Timestamp: ts}))
return encodedTS
}

// DecodeMVCCKey decodes an engine.MVCCKey from its serialized representation.
func DecodeMVCCKey(encodedKey []byte) (MVCCKey, error) {
k, ts, err := enginepb.DecodeKey(encodedKey)
return MVCCKey{k, ts}, err
}

// Decode the header of RocksDB batch repr, returning both the count of the
// entries in the batch and the suffix of data remaining in the batch.
func rocksDBBatchDecodeHeader(repr []byte) (count int, orepr pebble.BatchReader, err error) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/bench_pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -387,7 +387,7 @@ func BenchmarkBatchBuilderPut(b *testing.B) {
for j := i; j < end; j++ {
key := roachpb.Key(encoding.EncodeUvarintAscending(keyBuf[:4], uint64(j)))
ts := hlc.Timestamp{WallTime: int64(j)}
require.NoError(b, batch.Set(EncodeKey(MVCCKey{key, ts}), value, nil /* WriteOptions */))
require.NoError(b, batch.Set(EncodeMVCCKey(MVCCKey{key, ts}), value, nil /* WriteOptions */))
}
batch.Reset()
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/engine_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) {
}
for _, test := range testCases {
t.Run("", func(t *testing.T) {
encodedTS := encodeTimestamp(test.key.Timestamp)
encodedTS := encodeMVCCTimestamp(test.key.Timestamp)
eKey := EngineKey{Key: test.key.Key, Version: encodedTS}
b1 := eKey.Encode()
require.Equal(t, len(b1), eKey.EncodedLen())
Expand All @@ -116,7 +116,7 @@ func TestMVCCAndEngineKeyEncodeDecode(t *testing.T) {
keyPart, ok := GetKeyPartFromEngineKey(b2)
require.True(t, ok)
require.Equal(t, eKeyDecoded.Key, roachpb.Key(keyPart))
b3 := EncodeKey(test.key)
b3 := EncodeMVCCKey(test.key)
require.Equal(t, b3, b1)
k3, ts, ok := enginepb.SplitMVCCKey(b3)
require.True(t, ok)
Expand Down Expand Up @@ -168,7 +168,7 @@ func TestEngineKeyValidate(t *testing.T) {
case EngineKey:
ek = k
case MVCCKey:
ek = EngineKey{Key: k.Key, Version: encodeTimestamp(k.Timestamp)}
ek = EngineKey{Key: k.Key, Version: encodeMVCCTimestamp(k.Timestamp)}
case LockTableKey:
key, _ := k.ToEngineKey(nil)
ek = key
Expand Down
28 changes: 15 additions & 13 deletions pkg/storage/enginepb/decode.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,28 +43,30 @@ func SplitMVCCKey(mvccKey []byte) (key []byte, ts []byte, ok bool) {
}

// DecodeKey decodes an key/timestamp from its serialized representation.
func DecodeKey(encodedKey []byte) (key []byte, timestamp hlc.Timestamp, _ error) {
key, ts, ok := SplitMVCCKey(encodedKey)
func DecodeKey(encodedKey []byte) ([]byte, hlc.Timestamp, error) {
key, encodedTS, ok := SplitMVCCKey(encodedKey)
if !ok {
return nil, timestamp, errors.Errorf("invalid encoded mvcc key: %x", encodedKey)
return nil, hlc.Timestamp{}, errors.Errorf("invalid encoded mvcc key: %x", encodedKey)
}
switch len(ts) {
// NB: This logic is duplicated with storage.decodeMVCCTimestamp() to avoid the
// overhead of an additional function call (~13%).
var timestamp hlc.Timestamp
switch len(encodedTS) {
case 0:
// No-op.
case 8:
timestamp.WallTime = int64(binary.BigEndian.Uint64(ts[0:8]))
timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8]))
case 12:
timestamp.WallTime = int64(binary.BigEndian.Uint64(ts[0:8]))
timestamp.Logical = int32(binary.BigEndian.Uint32(ts[8:12]))
timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8]))
timestamp.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12]))
case 13:
timestamp.WallTime = int64(binary.BigEndian.Uint64(ts[0:8]))
timestamp.Logical = int32(binary.BigEndian.Uint32(ts[8:12]))
timestamp.Synthetic = ts[12] != 0
timestamp.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8]))
timestamp.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12]))
timestamp.Synthetic = encodedTS[12] != 0
default:
return nil, timestamp, errors.Errorf(
"invalid encoded mvcc key: %x bad timestamp %x", encodedKey, ts)
return nil, hlc.Timestamp{}, errors.Errorf(
"invalid encoded mvcc key: %x bad timestamp %x", encodedKey, encodedTS)
}

return key, timestamp, nil
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/enginepb/decode_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ func BenchmarkScanDecodeKeyValue(b *testing.B) {
ts := hlc.Timestamp{WallTime: int64(1000000)}
value := []byte("foo foo foo")
rep := make([]byte, 8)
keyBytes := storage.EncodeKey(storage.MVCCKey{Key: key, Timestamp: ts})
keyBytes := storage.EncodeMVCCKey(storage.MVCCKey{Key: key, Timestamp: ts})
binary.LittleEndian.PutUint64(rep, uint64(len(keyBytes)<<32)|uint64(len(value)))
rep = append(rep, keyBytes...)
rep = append(rep, value...)
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/intent_reader_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,7 +126,7 @@ var intentInterleavingReaderPool = sync.Pool{

// Get implements the Reader interface.
func (imr *intentInterleavingReader) MVCCGet(key MVCCKey) ([]byte, error) {
val, err := imr.wrappableReader.rawGet(EncodeKey(key))
val, err := imr.wrappableReader.rawGet(EncodeMVCCKey(key))
if val != nil || err != nil || !key.Timestamp.IsEmpty() {
return val, err
}
Expand Down
Loading

0 comments on commit c4f15d6

Please sign in to comment.