diff --git a/pkg/cli/debug.go b/pkg/cli/debug.go index 9f4267c03879..08a9cdc9d082 100644 --- a/pkg/cli/debug.go +++ b/pkg/cli/debug.go @@ -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 { diff --git a/pkg/cli/flags_util.go b/pkg/cli/flags_util.go index 4b31f81c5320..2117cd2766fb 100644 --- a/pkg/cli/flags_util.go +++ b/pkg/cli/flags_util.go @@ -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) diff --git a/pkg/kv/kvnemesis/engine.go b/pkg/kv/kvnemesis/engine.go index e19ec8665c0c..f6cc5391c789 100644 --- a/pkg/kv/kvnemesis/engine.go +++ b/pkg/kv/kvnemesis/engine.go @@ -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{} } @@ -78,7 +78,7 @@ 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) } } @@ -86,7 +86,7 @@ func (e *Engine) Put(key storage.MVCCKey, value []byte) { // 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) } } diff --git a/pkg/kv/kvnemesis/validator.go b/pkg/kv/kvnemesis/validator.go index 4a5c8279328a..b312a85d5d56 100644 --- a/pkg/kv/kvnemesis/validator.go +++ b/pkg/kv/kvnemesis/validator.go @@ -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) } } @@ -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: @@ -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 { @@ -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 { diff --git a/pkg/kv/kvserver/debug_print.go b/pkg/kv/kvserver/debug_print.go index 4d5fac25d5a6..31a5aeee2f36 100644 --- a/pkg/kv/kvserver/debug_print.go +++ b/pkg/kv/kvserver/debug_print.go @@ -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 diff --git a/pkg/kv/kvserver/debug_print_test.go b/pkg/kv/kvserver/debug_print_test.go index 05563d5cec1c..868f0897a9d3 100644 --- a/pkg/kv/kvserver/debug_print_test.go +++ b/pkg/kv/kvserver/debug_print_test.go @@ -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 */)) diff --git a/pkg/kv/kvserver/replica_consistency_diff.go b/pkg/kv/kvserver/replica_consistency_diff.go index 70e59f56a1da..8d132fa31e37 100644 --- a/pkg/kv/kvserver/replica_consistency_diff.go +++ b/pkg/kv/kvserver/replica_consistency_diff.go @@ -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) } } diff --git a/pkg/storage/BUILD.bazel b/pkg/storage/BUILD.bazel index f567e3db8b4f..5952a8b79c08 100644 --- a/pkg/storage/BUILD.bazel +++ b/pkg/storage/BUILD.bazel @@ -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", @@ -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", diff --git a/pkg/storage/batch.go b/pkg/storage/batch.go index 85aecdacd192..3473967a9d95 100644 --- a/pkg/storage/batch.go +++ b/pkg/storage/batch.go @@ -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" ) @@ -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) { diff --git a/pkg/storage/bench_pebble_test.go b/pkg/storage/bench_pebble_test.go index 5639222f6780..cdb94dd8247d 100644 --- a/pkg/storage/bench_pebble_test.go +++ b/pkg/storage/bench_pebble_test.go @@ -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() } diff --git a/pkg/storage/engine_key_test.go b/pkg/storage/engine_key_test.go index 0d85a50f081d..80703857e117 100644 --- a/pkg/storage/engine_key_test.go +++ b/pkg/storage/engine_key_test.go @@ -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()) @@ -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) @@ -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 diff --git a/pkg/storage/enginepb/decode.go b/pkg/storage/enginepb/decode.go index 5857ed34f270..8b876dd79587 100644 --- a/pkg/storage/enginepb/decode.go +++ b/pkg/storage/enginepb/decode.go @@ -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 } diff --git a/pkg/storage/enginepb/decode_test.go b/pkg/storage/enginepb/decode_test.go index b4437cb35a19..f1ae35de8b88 100644 --- a/pkg/storage/enginepb/decode_test.go +++ b/pkg/storage/enginepb/decode_test.go @@ -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...) diff --git a/pkg/storage/intent_reader_writer.go b/pkg/storage/intent_reader_writer.go index 956dc0a7f52e..9e6068f8006d 100644 --- a/pkg/storage/intent_reader_writer.go +++ b/pkg/storage/intent_reader_writer.go @@ -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 } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 4216ac146663..64b0d1abaef2 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -57,14 +57,6 @@ const ( maxIntentsPerWriteIntentErrorDefault = 5000 ) -var ( - // MVCCKeyMax is a maximum mvcc-encoded key value which sorts after - // all other keys. - MVCCKeyMax = MakeMVCCMetadataKey(roachpb.KeyMax) - // NilKey is the nil MVCCKey. - NilKey = MVCCKey{} -) - var minWALSyncInterval = settings.RegisterDurationSetting( settings.TenantWritable, "rocksdb.min_wal_sync_interval", @@ -101,107 +93,6 @@ func emptyKeyError() error { return errors.Errorf("attempted access to empty key") } -// MVCCKey is a versioned key, distinguished from roachpb.Key with the addition -// of a timestamp. -type MVCCKey struct { - Key roachpb.Key - Timestamp hlc.Timestamp -} - -// MakeMVCCMetadataKey creates an MVCCKey from a roachpb.Key. -func MakeMVCCMetadataKey(key roachpb.Key) MVCCKey { - return MVCCKey{Key: key} -} - -// Next returns the next key. -func (k MVCCKey) Next() MVCCKey { - ts := k.Timestamp.Prev() - if ts.IsEmpty() { - return MVCCKey{ - Key: k.Key.Next(), - } - } - return MVCCKey{ - Key: k.Key, - Timestamp: ts, - } -} - -// Less compares two keys. -func (k MVCCKey) Less(l MVCCKey) bool { - if c := k.Key.Compare(l.Key); c != 0 { - return c < 0 - } - if !k.IsValue() { - return l.IsValue() - } else if !l.IsValue() { - return false - } - return l.Timestamp.Less(k.Timestamp) -} - -// Equal returns whether two keys are identical. -func (k MVCCKey) Equal(l MVCCKey) bool { - return k.Key.Compare(l.Key) == 0 && k.Timestamp.EqOrdering(l.Timestamp) -} - -// IsValue returns true iff the timestamp is non-zero. -func (k MVCCKey) IsValue() bool { - return !k.Timestamp.IsEmpty() -} - -// EncodedSize returns the size of the MVCCKey when encoded. -func (k MVCCKey) EncodedSize() int { - n := len(k.Key) + 1 - if k.IsValue() { - // Note that this isn't quite accurate: timestamps consume between 8-13 - // bytes. Fixing this only adjusts the accounting for timestamps, not the - // actual on disk storage. - n += int(MVCCVersionTimestampSize) - } - return n -} - -// String returns a string-formatted version of the key. -func (k MVCCKey) String() string { - if !k.IsValue() { - return k.Key.String() - } - return fmt.Sprintf("%s/%s", k.Key, k.Timestamp) -} - -// Format implements the fmt.Formatter interface. -func (k MVCCKey) Format(f fmt.State, c rune) { - fmt.Fprintf(f, "%s/%s", k.Key, k.Timestamp) -} - -// Len returns the size of the MVCCKey when encoded. Implements the -// pebble.Encodeable interface. -// -// TODO(itsbilal): Reconcile this with EncodedSize. Would require updating MVCC -// stats tests to reflect the more accurate lengths provided by this function. -func (k MVCCKey) Len() int { - const ( - timestampSentinelLen = 1 - walltimeEncodedLen = 8 - logicalEncodedLen = 4 - syntheticEncodedLen = 1 - timestampEncodedLengthLen = 1 - ) - - n := len(k.Key) + timestampEncodedLengthLen - if !k.Timestamp.IsEmpty() { - n += timestampSentinelLen + walltimeEncodedLen - if k.Timestamp.Logical != 0 || k.Timestamp.Synthetic { - n += logicalEncodedLen - } - if k.Timestamp.Synthetic { - n += syntheticEncodedLen - } - } - return n -} - // MVCCKeyValue contains the raw bytes of the value for a key. type MVCCKeyValue struct { Key MVCCKey diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go new file mode 100644 index 000000000000..02cb4016ba92 --- /dev/null +++ b/pkg/storage/mvcc_key.go @@ -0,0 +1,289 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "encoding/binary" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/errors" +) + +var ( + // MVCCKeyMax sorts after all other MVCC keys. + MVCCKeyMax = MakeMVCCMetadataKey(roachpb.KeyMax) + // NilKey is the nil MVCCKey. + NilKey = MVCCKey{} +) + +const ( + mvccEncodedTimeSentinelLen = 1 + mvccEncodedTimeWallLen = 8 + mvccEncodedTimeLogicalLen = 4 + mvccEncodedTimeSyntheticLen = 1 + mvccEncodedTimeLengthLen = 1 +) + +// MVCCKey is a versioned key, distinguished from roachpb.Key with the addition +// of a timestamp. +type MVCCKey struct { + Key roachpb.Key + Timestamp hlc.Timestamp +} + +// MakeMVCCMetadataKey creates an MVCCKey from a roachpb.Key. +func MakeMVCCMetadataKey(key roachpb.Key) MVCCKey { + return MVCCKey{Key: key} +} + +// Next returns the next key. +func (k MVCCKey) Next() MVCCKey { + ts := k.Timestamp.Prev() + if ts.IsEmpty() { + return MVCCKey{ + Key: k.Key.Next(), + } + } + return MVCCKey{ + Key: k.Key, + Timestamp: ts, + } +} + +// Less compares two keys. +func (k MVCCKey) Less(l MVCCKey) bool { + if c := k.Key.Compare(l.Key); c != 0 { + return c < 0 + } + if !k.IsValue() { + return l.IsValue() + } else if !l.IsValue() { + return false + } + return l.Timestamp.Less(k.Timestamp) +} + +// Equal returns whether two keys are identical. +func (k MVCCKey) Equal(l MVCCKey) bool { + return k.Key.Compare(l.Key) == 0 && k.Timestamp.EqOrdering(l.Timestamp) +} + +// IsValue returns true iff the timestamp is non-zero. +func (k MVCCKey) IsValue() bool { + return !k.Timestamp.IsEmpty() +} + +// EncodedSize returns the size of the MVCCKey when encoded. +// +// TODO(itsbilal): Reconcile this with Len(). Would require updating MVCC stats +// tests to reflect the more accurate lengths provided by Len(). +func (k MVCCKey) EncodedSize() int { + n := len(k.Key) + 1 + if k.IsValue() { + // Note that this isn't quite accurate: timestamps consume between 8-13 + // bytes. Fixing this only adjusts the accounting for timestamps, not the + // actual on disk storage. + n += int(MVCCVersionTimestampSize) + } + return n +} + +// String returns a string-formatted version of the key. +func (k MVCCKey) String() string { + if !k.IsValue() { + return k.Key.String() + } + return fmt.Sprintf("%s/%s", k.Key, k.Timestamp) +} + +// Format implements the fmt.Formatter interface. +func (k MVCCKey) Format(f fmt.State, c rune) { + fmt.Fprintf(f, "%s/%s", k.Key, k.Timestamp) +} + +// Len returns the size of the MVCCKey when encoded. Implements the +// pebble.Encodeable interface. +func (k MVCCKey) Len() int { + return encodedMVCCKeyLength(k) +} + +// EncodeMVCCKey encodes an MVCCKey into its Pebble representation. The encoding +// takes the following forms, where trailing time components are omitted when +// zero-valued: +// +// [key] [sentinel] [timeWall] [timeLogical] [timeSynthetic] [timeLength] +// [key] [sentinel] [timeWall] [timeLogical] [timeLength] +// [key] [sentinel] [timeWall] [timeLength] +// [key] [sentinel] +// +// key: the unmodified binary key (variable length) +// sentinel: separates key and timestamp (1 byte: 0x00) +// timeWall: Timestamp.WallTime (8 bytes: big-endian uint64) +// timeLogical: Timestamp.Logical (4 bytes: big-endian uint32) +// timeSynthetic: Timestamp.Synthetic (1 byte: 0x01 when set) +// timeLength: encoded timestamp length inc. itself (1 byte: uint8) +// +// The sentinel byte can be used to detect a key without a timestamp, since +// timeLength will never be 0 (it includes itself in the length). +func EncodeMVCCKey(key MVCCKey) []byte { + keyLen := encodedMVCCKeyLength(key) + buf := make([]byte, keyLen) + encodeMVCCKeyToBuf(buf, key, keyLen) + return buf +} + +// EncodeMVCCKeyToBuf encodes an MVCCKey into its Pebble representation, reusing +// the given byte buffer if it has sufficient capacity. +func EncodeMVCCKeyToBuf(buf []byte, key MVCCKey) []byte { + keyLen := encodedMVCCKeyLength(key) + if cap(buf) < keyLen { + buf = make([]byte, keyLen) + } else { + buf = buf[:keyLen] + } + encodeMVCCKeyToBuf(buf, key, keyLen) + return buf +} + +// encodeMVCCKeyToBuf encodes an MVCCKey into its Pebble representation to the +// target buffer, which must have the correct size. +func encodeMVCCKeyToBuf(buf []byte, key MVCCKey, keyLen int) { + copy(buf, key.Key) + pos := len(key.Key) + + buf[pos] = 0 // sentinel byte + pos += mvccEncodedTimeSentinelLen + + tsLen := keyLen - pos - mvccEncodedTimeLengthLen + if tsLen > 0 { + encodeMVCCTimestampToBuf(buf[pos:], key.Timestamp) + pos += tsLen + buf[pos] = byte(tsLen + mvccEncodedTimeLengthLen) + } +} + +// encodeMVCCTimestamp encodes an MVCC timestamp into its Pebble +// representation, excluding length suffix and sentinel byte. +func encodeMVCCTimestamp(ts hlc.Timestamp) []byte { + tsLen := encodedMVCCTimestampLength(ts) + if tsLen == 0 { + return nil + } + buf := make([]byte, tsLen) + encodeMVCCTimestampToBuf(buf, ts) + return buf +} + +// encodeMVCCTimestampSuffix encodes an MVCC timestamp into its Pebble +// representation, including the length suffix but excluding the sentinel byte. +// This is equivalent to the Pebble suffix. +func encodeMVCCTimestampSuffix(ts hlc.Timestamp) []byte { + tsLen := encodedMVCCTimestampLength(ts) + if tsLen == 0 { + return nil + } + buf := make([]byte, tsLen+mvccEncodedTimeLengthLen) + encodeMVCCTimestampToBuf(buf, ts) + buf[tsLen] = byte(tsLen + mvccEncodedTimeLengthLen) + return buf +} + +// encodeMVCCTimestampToBuf encodes an MVCC timestamp into its Pebble +// representation, excluding the length suffix and sentinel byte. The target +// buffer must have the correct size, and the timestamp must not be empty. +func encodeMVCCTimestampToBuf(buf []byte, ts hlc.Timestamp) { + binary.BigEndian.PutUint64(buf, uint64(ts.WallTime)) + if ts.Logical != 0 || ts.Synthetic { + binary.BigEndian.PutUint32(buf[mvccEncodedTimeWallLen:], uint32(ts.Logical)) + if ts.Synthetic { + buf[mvccEncodedTimeWallLen+mvccEncodedTimeLogicalLen] = 1 + } + } +} + +// encodedMVCCKeyLength returns the encoded length of the given MVCCKey. +func encodedMVCCKeyLength(key MVCCKey) int { + keyLen := len(key.Key) + mvccEncodedTimeSentinelLen + if !key.Timestamp.IsEmpty() { + keyLen += mvccEncodedTimeWallLen + mvccEncodedTimeLengthLen + if key.Timestamp.Logical != 0 || key.Timestamp.Synthetic { + keyLen += mvccEncodedTimeLogicalLen + if key.Timestamp.Synthetic { + keyLen += mvccEncodedTimeSyntheticLen + } + } + } + return keyLen +} + +// encodedMVCCTimestampLength returns the encoded length of the given MVCC +// timestamp, excluding the length suffix and sentinel bytes. +func encodedMVCCTimestampLength(ts hlc.Timestamp) int { + // This is backwards, but encodedMVCCKeyLength() is called in the + // EncodeMVCCKey() hot path and an additional function call to this function + // shows ~6% overhead in benchmarks. We therefore do the timestamp length + // calculation inline in encodedMVCCKeyLength(), and remove the excess here. + tsLen := encodedMVCCKeyLength(MVCCKey{Timestamp: ts}) - mvccEncodedTimeSentinelLen + if tsLen > 0 { + tsLen -= mvccEncodedTimeLengthLen + } + return tsLen +} + +// TODO(erikgrinaker): merge in the enginepb decoding functions once it can +// avoid the storage package's problematic CGo dependency (via Pebble). + +// DecodeMVCCKey decodes an MVCCKey from its Pebble representation. +func DecodeMVCCKey(encodedKey []byte) (MVCCKey, error) { + k, ts, err := enginepb.DecodeKey(encodedKey) + return MVCCKey{k, ts}, err +} + +// decodeMVCCTimestamp decodes an MVCC timestamp from its Pebble representation, +// excluding the length suffix. +func decodeMVCCTimestamp(encodedTS []byte) (hlc.Timestamp, error) { + // NB: This logic is duplicated in enginepb.DecodeKey() to avoid the + // overhead of an additional function call there (~13%). + var ts hlc.Timestamp + switch len(encodedTS) { + case 0: + // No-op. + case 8: + ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + case 12: + ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + case 13: + ts.WallTime = int64(binary.BigEndian.Uint64(encodedTS[0:8])) + ts.Logical = int32(binary.BigEndian.Uint32(encodedTS[8:12])) + ts.Synthetic = encodedTS[12] != 0 + default: + return hlc.Timestamp{}, errors.Errorf("bad timestamp %x", encodedTS) + } + return ts, nil +} + +// decodeMVCCTimestampSuffix decodes an MVCC timestamp from its Pebble representation, +// including the length suffix. +func decodeMVCCTimestampSuffix(encodedTS []byte) (hlc.Timestamp, error) { + if len(encodedTS) == 0 { + return hlc.Timestamp{}, nil + } + encodedLen := len(encodedTS) + if suffixLen := int(encodedTS[encodedLen-1]); suffixLen != encodedLen { + return hlc.Timestamp{}, errors.Errorf( + "bad timestamp: found length suffix %d, actual length %d", suffixLen, encodedLen) + } + return decodeMVCCTimestamp(encodedTS[:encodedLen-1]) +} diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go new file mode 100644 index 000000000000..63af9beda842 --- /dev/null +++ b/pkg/storage/mvcc_key_test.go @@ -0,0 +1,240 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package storage + +import ( + "bytes" + "encoding/hex" + "fmt" + "math" + "reflect" + "sort" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/shuffle" + "github.com/stretchr/testify/require" +) + +// Verify the sort ordering of successive keys with metadata and +// versioned values. In particular, the following sequence of keys / +// versions: +// +// a +// a +// a +// a +// a\x00 +// a\x00 +// a\x00 +// a\x00 +func TestMVCCKeys(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + aKey := roachpb.Key("a") + a0Key := roachpb.Key("a\x00") + keys := mvccKeys{ + mvccKey(aKey), + mvccVersionKey(aKey, hlc.Timestamp{WallTime: math.MaxInt64}), + mvccVersionKey(aKey, hlc.Timestamp{WallTime: 1}), + mvccVersionKey(aKey, hlc.Timestamp{Logical: 1}), + mvccKey(a0Key), + mvccVersionKey(a0Key, hlc.Timestamp{WallTime: math.MaxInt64}), + mvccVersionKey(a0Key, hlc.Timestamp{WallTime: 1}), + mvccVersionKey(a0Key, hlc.Timestamp{Logical: 1}), + } + sortKeys := make(mvccKeys, len(keys)) + copy(sortKeys, keys) + shuffle.Shuffle(sortKeys) + sort.Sort(sortKeys) + if !reflect.DeepEqual(sortKeys, keys) { + t.Errorf("expected keys to sort in order %s, but got %s", keys, sortKeys) + } +} + +func TestEncodeDecodeMVCCKeyAndTimestamp(t *testing.T) { + defer leaktest.AfterTest(t)() + + testcases := map[string]struct { + key string + ts hlc.Timestamp + encoded string // hexadecimal + }{ + "empty": {"", hlc.Timestamp{}, "00"}, + "only key": {"foo", hlc.Timestamp{}, "666f6f00"}, + "no key": {"", hlc.Timestamp{WallTime: 1643550788737652545}, "0016cf10bc0505574109"}, + "walltime": {"foo", hlc.Timestamp{WallTime: 1643550788737652545}, "666f6f0016cf10bc0505574109"}, + "logical": {"foo", hlc.Timestamp{Logical: 65535}, "666f6f0000000000000000000000ffff0d"}, + "synthetic": {"foo", hlc.Timestamp{Synthetic: true}, "666f6f00000000000000000000000000010e"}, + "walltime and logical": {"foo", hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535}, "666f6f0016cf10bc050557410000ffff0d"}, + "walltime and synthetic": {"foo", hlc.Timestamp{WallTime: 1643550788737652545, Synthetic: true}, "666f6f0016cf10bc0505574100000000010e"}, + "logical and synthetic": {"foo", hlc.Timestamp{Logical: 65535, Synthetic: true}, "666f6f0000000000000000000000ffff010e"}, + "all": {"foo", hlc.Timestamp{WallTime: 1643550788737652545, Logical: 65535, Synthetic: true}, "666f6f0016cf10bc050557410000ffff010e"}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + + // Test Encode/DecodeMVCCKey. + expect, err := hex.DecodeString(tc.encoded) + require.NoError(t, err) + if len(expect) == 0 { + expect = nil + } + + mvccKey := MVCCKey{Key: []byte(tc.key), Timestamp: tc.ts} + + encoded := EncodeMVCCKey(mvccKey) + require.Equal(t, expect, encoded) + + decoded, err := DecodeMVCCKey(encoded) + require.NoError(t, err) + require.Equal(t, mvccKey, decoded) + + // Test encode/decodeMVCCTimestampSuffix too, since we can trivially do so. + expectTS, err := hex.DecodeString(tc.encoded[2*len(tc.key)+2:]) + require.NoError(t, err) + if len(expectTS) == 0 { + expectTS = nil + } + + encodedTS := encodeMVCCTimestampSuffix(tc.ts) + require.Equal(t, expectTS, encodedTS) + + decodedTS, err := decodeMVCCTimestampSuffix(encodedTS) + require.NoError(t, err) + require.Equal(t, tc.ts, decodedTS) + + // Test encode/decodeMVCCTimestamp as well, for completeness. + if len(expectTS) > 0 { + expectTS = expectTS[:len(expectTS)-1] + } + + encodedTS = encodeMVCCTimestamp(tc.ts) + require.Equal(t, expectTS, encodedTS) + + decodedTS, err = decodeMVCCTimestamp(encodedTS) + require.NoError(t, err) + require.Equal(t, tc.ts, decodedTS) + }) + } +} + +func TestDecodeMVCCKeyErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + + testcases := map[string]struct { + encoded string // hex-encoded + expectErr string + }{ + "empty input": {"", "invalid encoded mvcc key: "}, + "lone length suffix": {"01", "invalid encoded mvcc key: "}, + "invalid timestamp length": {"ab00ffff03", "invalid encoded mvcc key: ab00ffff03 bad timestamp ffff"}, + "invalid timestamp length suffix": {"ab00ffffffffffffffff0f", "invalid encoded mvcc key: ab00ffffffffffffffff0f"}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + encoded, err := hex.DecodeString(tc.encoded) + require.NoError(t, err) + + _, err = DecodeMVCCKey(encoded) + require.Error(t, err) + require.Contains(t, err.Error(), tc.expectErr) + }) + } +} + +func TestDecodeMVCCTimestampSuffixErrors(t *testing.T) { + defer leaktest.AfterTest(t)() + + testcases := map[string]struct { + encoded string // hex-encoded + expectErr string + }{ + "invalid length": {"ffff03", "bad timestamp ffff"}, + "invalid length suffix": {"ffffffffffffffff0f", "bad timestamp: found length suffix 15, actual length 9"}, + } + for name, tc := range testcases { + tc := tc + t.Run(name, func(t *testing.T) { + encoded, err := hex.DecodeString(tc.encoded) + require.NoError(t, err) + + _, err = decodeMVCCTimestampSuffix(encoded) + require.Error(t, err) + require.Contains(t, err.Error(), tc.expectErr) + }) + } +} + +var benchmarkEncodeMVCCKeyResult []byte + +func BenchmarkEncodeMVCCKey(b *testing.B) { + keys := map[string][]byte{ + "empty": {}, + "short": []byte("foo"), + "long": bytes.Repeat([]byte{1}, 4096), + } + timestamps := map[string]hlc.Timestamp{ + "empty": {}, + "walltime": {WallTime: 1643550788737652545}, + "walltime+logical": {WallTime: 1643550788737652545, Logical: 4096}, + "all": {WallTime: 1643550788737652545, Logical: 4096, Synthetic: true}, + } + buf := make([]byte, 0, 65536) + for keyDesc, key := range keys { + for tsDesc, ts := range timestamps { + mvccKey := MVCCKey{Key: key, Timestamp: ts} + b.Run(fmt.Sprintf("key=%s/ts=%s", keyDesc, tsDesc), func(b *testing.B) { + for i := 0; i < b.N; i++ { + EncodeMVCCKeyToBuf(buf, mvccKey) + } + }) + } + } + benchmarkEncodeMVCCKeyResult = buf // avoid compiler optimizing away function call +} + +var benchmarkDecodeMVCCKeyResult MVCCKey + +func BenchmarkDecodeMVCCKey(b *testing.B) { + keys := map[string][]byte{ + "empty": {}, + "short": []byte("foo"), + "long": bytes.Repeat([]byte{1}, 4096), + } + timestamps := map[string]hlc.Timestamp{ + "empty": {}, + "walltime": {WallTime: 1643550788737652545}, + "walltime+logical": {WallTime: 1643550788737652545, Logical: 4096}, + "all": {WallTime: 1643550788737652545, Logical: 4096, Synthetic: true}, + } + var mvccKey MVCCKey + var err error + for keyDesc, key := range keys { + for tsDesc, ts := range timestamps { + encoded := EncodeMVCCKey(MVCCKey{Key: key, Timestamp: ts}) + b.Run(fmt.Sprintf("key=%s/ts=%s", keyDesc, tsDesc), func(b *testing.B) { + for i := 0; i < b.N; i++ { + mvccKey, err = DecodeMVCCKey(encoded) + if err != nil { // for performance + require.NoError(b, err) + } + } + }) + } + } + benchmarkDecodeMVCCKeyResult = mvccKey // avoid compiler optimizing away function call +} diff --git a/pkg/storage/mvcc_test.go b/pkg/storage/mvcc_test.go index ad98bc972860..72dfa9114e2d 100644 --- a/pkg/storage/mvcc_test.go +++ b/pkg/storage/mvcc_test.go @@ -37,7 +37,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/randutil" - "github.com/cockroachdb/cockroach/pkg/util/shuffle" "github.com/cockroachdb/cockroach/pkg/util/uuid" "github.com/cockroachdb/errors" "github.com/kr/pretty" @@ -250,42 +249,6 @@ func TestMVCCStatsAddSubForward(t *testing.T) { cmp(neg, exp) } -// Verify the sort ordering of successive keys with metadata and -// versioned values. In particular, the following sequence of keys / -// versions: -// -// a -// a -// a -// a -// a\x00 -// a\x00 -// a\x00 -// a\x00 -func TestMVCCKeys(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - aKey := roachpb.Key("a") - a0Key := roachpb.Key("a\x00") - keys := mvccKeys{ - mvccKey(aKey), - mvccVersionKey(aKey, hlc.Timestamp{WallTime: math.MaxInt64}), - mvccVersionKey(aKey, hlc.Timestamp{WallTime: 1}), - mvccVersionKey(aKey, hlc.Timestamp{Logical: 1}), - mvccKey(a0Key), - mvccVersionKey(a0Key, hlc.Timestamp{WallTime: math.MaxInt64}), - mvccVersionKey(a0Key, hlc.Timestamp{WallTime: 1}), - mvccVersionKey(a0Key, hlc.Timestamp{Logical: 1}), - } - sortKeys := make(mvccKeys, len(keys)) - copy(sortKeys, keys) - shuffle.Shuffle(sortKeys) - sort.Sort(sortKeys) - if !reflect.DeepEqual(sortKeys, keys) { - t.Errorf("expected keys to sort in order %s, but got %s", keys, sortKeys) - } -} - func TestMVCCGetNotExist(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index 1fb6c0a787e7..33af2afaa65d 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -295,7 +295,7 @@ func (t *pebbleTimeBoundPropCollector) Finish(userProps map[string]string) error return nil //nolint:returnerrcheck } if meta.Txn != nil { - ts := encodeTimestamp(meta.Timestamp.ToTimestamp()) + ts := encodeMVCCTimestamp(meta.Timestamp.ToTimestamp()) t.updateBounds(ts) } } @@ -1052,7 +1052,7 @@ func (p *Pebble) clear(key MVCCKey) error { if len(key.Key) == 0 { return emptyKeyError() } - return p.db.Delete(EncodeKey(key), pebble.Sync) + return p.db.Delete(EncodeMVCCKey(key), pebble.Sync) } // SingleClearEngineKey implements the Engine interface. @@ -1081,8 +1081,8 @@ func (p *Pebble) ClearMVCCRange(start, end MVCCKey) error { } func (p *Pebble) clearRange(start, end MVCCKey) error { - bufStart := EncodeKey(start) - bufEnd := EncodeKey(end) + bufStart := EncodeMVCCKey(start) + bufEnd := EncodeMVCCKey(end) return p.db.DeleteRange(bufStart, bufEnd, pebble.Sync) } @@ -1103,7 +1103,7 @@ func (p *Pebble) Merge(key MVCCKey, value []byte) error { if len(key.Key) == 0 { return emptyKeyError() } - return p.db.Merge(EncodeKey(key), value, pebble.Sync) + return p.db.Merge(EncodeMVCCKey(key), value, pebble.Sync) } // PutMVCC implements the Engine interface. @@ -1139,7 +1139,7 @@ func (p *Pebble) put(key MVCCKey, value []byte) error { if len(key.Key) == 0 { return emptyKeyError() } - return p.db.Set(EncodeKey(key), value, pebble.Sync) + return p.db.Set(EncodeMVCCKey(key), value, pebble.Sync) } // LogData implements the Engine interface. @@ -1435,13 +1435,13 @@ func (p *Pebble) ApproximateDiskBytes(from, to roachpb.Key) (uint64, error) { // Compact implements the Engine interface. func (p *Pebble) Compact() error { - return p.db.Compact(nil, EncodeKey(MVCCKeyMax)) + return p.db.Compact(nil, EncodeMVCCKey(MVCCKeyMax)) } // CompactRange implements the Engine interface. func (p *Pebble) CompactRange(start, end roachpb.Key, forceBottommost bool) error { - bufStart := EncodeKey(MVCCKey{start, hlc.Timestamp{}}) - bufEnd := EncodeKey(MVCCKey{end, hlc.Timestamp{}}) + bufStart := EncodeMVCCKey(MVCCKey{start, hlc.Timestamp{}}) + bufEnd := EncodeMVCCKey(MVCCKey{end, hlc.Timestamp{}}) return p.db.Compact(bufStart, bufEnd) } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index f2e4847e8e38..117c2995cfa8 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -349,7 +349,7 @@ func (p *pebbleBatch) clear(key MVCCKey) error { return emptyKeyError() } - p.buf = EncodeKeyToBuf(p.buf[:0], key) + p.buf = EncodeMVCCKeyToBuf(p.buf[:0], key) return p.batch.Delete(p.buf, nil) } @@ -381,8 +381,8 @@ func (p *pebbleBatch) ClearMVCCRange(start, end MVCCKey) error { } func (p *pebbleBatch) clearRange(start, end MVCCKey) error { - p.buf = EncodeKeyToBuf(p.buf[:0], start) - buf2 := EncodeKey(end) + p.buf = EncodeMVCCKeyToBuf(p.buf[:0], start) + buf2 := EncodeMVCCKey(end) return p.batch.DeleteRange(p.buf, buf2, nil) } @@ -419,7 +419,7 @@ func (p *pebbleBatch) Merge(key MVCCKey, value []byte) error { return emptyKeyError() } - p.buf = EncodeKeyToBuf(p.buf[:0], key) + p.buf = EncodeMVCCKeyToBuf(p.buf[:0], key) return p.batch.Merge(p.buf, value, nil) } @@ -460,7 +460,7 @@ func (p *pebbleBatch) put(key MVCCKey, value []byte) error { return emptyKeyError() } - p.buf = EncodeKeyToBuf(p.buf[:0], key) + p.buf = EncodeMVCCKeyToBuf(p.buf[:0], key) return p.batch.Set(p.buf, value, nil) } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index dcd439abd06b..cc4a5ebe502e 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -131,8 +131,8 @@ func (p *pebbleIterator) init(handle pebble.Reader, iterToClone cloneableIter, o doClone := iterToClone != nil if !opts.MaxTimestampHint.IsEmpty() { doClone = false - encodedMinTS := string(encodeTimestamp(opts.MinTimestampHint)) - encodedMaxTS := string(encodeTimestamp(opts.MaxTimestampHint)) + encodedMinTS := string(encodeMVCCTimestamp(opts.MinTimestampHint)) + encodedMaxTS := string(encodeMVCCTimestamp(opts.MaxTimestampHint)) p.options.TableFilter = func(userProps map[string]string) bool { tableMinTS := userProps["crdb.ts.min"] if len(tableMinTS) == 0 { @@ -267,7 +267,7 @@ func (p *pebbleIterator) Close() { func (p *pebbleIterator) SeekGE(key MVCCKey) { p.mvccDirIsReverse = false p.mvccDone = false - p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], key) + p.keyBuf = EncodeMVCCKeyToBuf(p.keyBuf[:0], key) if p.prefix { p.iter.SeekPrefixGE(p.keyBuf) } else { @@ -471,7 +471,7 @@ func (p *pebbleIterator) UnsafeValue() []byte { func (p *pebbleIterator) SeekLT(key MVCCKey) { p.mvccDirIsReverse = true p.mvccDone = false - p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], key) + p.keyBuf = EncodeMVCCKeyToBuf(p.keyBuf[:0], key) p.iter.SeekLT(p.keyBuf) } diff --git a/pkg/storage/pebble_mvcc_scanner.go b/pkg/storage/pebble_mvcc_scanner.go index 6fa0f3910393..2a9904aecaa8 100644 --- a/pkg/storage/pebble_mvcc_scanner.go +++ b/pkg/storage/pebble_mvcc_scanner.go @@ -743,7 +743,7 @@ func (p *pebbleMVCCScanner) getAndAdvance(ctx context.Context) bool { // about to advance. If this proves to be a problem later, we can extend // addAndAdvance to take an MVCCKey explicitly. p.curUnsafeKey.Timestamp = metaTS - p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], p.curUnsafeKey) + p.keyBuf = EncodeMVCCKeyToBuf(p.keyBuf[:0], p.curUnsafeKey) return p.addAndAdvance(ctx, p.curUnsafeKey.Key, p.keyBuf, value) } // 13. If no value in the intent history has a sequence number equal to @@ -971,7 +971,7 @@ func (p *pebbleMVCCScanner) seekVersion( ctx context.Context, seekTS hlc.Timestamp, uncertaintyCheck bool, ) bool { seekKey := MVCCKey{Key: p.curUnsafeKey.Key, Timestamp: seekTS} - p.keyBuf = EncodeKeyToBuf(p.keyBuf[:0], seekKey) + p.keyBuf = EncodeMVCCKeyToBuf(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 // MVCCIterator will be at a different version of the same key, it is free diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 78fd4879a51f..48f5c9533588 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -64,17 +64,17 @@ func TestEngineComparer(t *testing.T) { Timestamp: hlc.Timestamp{WallTime: 2}, } - require.Equal(t, -1, EngineComparer.Compare(EncodeKey(keyAMetadata), EncodeKey(keyA1)), + require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyAMetadata), EncodeMVCCKey(keyA1)), "expected key metadata to sort first") - require.Equal(t, -1, EngineComparer.Compare(EncodeKey(keyA2), EncodeKey(keyA1)), + require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyA2), EncodeMVCCKey(keyA1)), "expected higher timestamp to sort first") - require.Equal(t, -1, EngineComparer.Compare(EncodeKey(keyA2), EncodeKey(keyB2)), + require.Equal(t, -1, EngineComparer.Compare(EncodeMVCCKey(keyA2), EncodeMVCCKey(keyB2)), "expected lower key to sort first") suffix := func(key []byte) []byte { return key[EngineComparer.Split(key):] } - require.Equal(t, -1, EngineComparer.Compare(suffix(EncodeKey(keyA2)), suffix(EncodeKey(keyA1))), + require.Equal(t, -1, EngineComparer.Compare(suffix(EncodeMVCCKey(keyA2)), suffix(EncodeMVCCKey(keyA1))), "expected bare suffix with higher timestamp to sort first") } @@ -102,7 +102,7 @@ func TestPebbleTimeBoundPropCollector(t *testing.T) { return err.Error() } ikey := pebble.InternalKey{ - UserKey: EncodeKey(MVCCKey{ + UserKey: EncodeMVCCKey(MVCCKey{ Key: key, Timestamp: hlc.Timestamp{WallTime: int64(timestamp)}, }), @@ -416,8 +416,8 @@ func TestPebbleSeparatorSuccessor(t *testing.T) { } for _, tc := range sepCases { t.Run("", func(t *testing.T) { - got := string(EngineComparer.Separator(nil, EncodeKey(tc.a), EncodeKey(tc.b))) - if got != string(EncodeKey(tc.want)) { + got := string(EngineComparer.Separator(nil, EncodeMVCCKey(tc.a), EncodeMVCCKey(tc.b))) + if got != string(EncodeMVCCKey(tc.want)) { t.Errorf("a, b = %q, %q: got %q, want %q", tc.a, tc.b, got, tc.want) } }) @@ -451,8 +451,8 @@ func TestPebbleSeparatorSuccessor(t *testing.T) { } for _, tc := range succCases { t.Run("", func(t *testing.T) { - got := string(EngineComparer.Successor(nil, EncodeKey(tc.a))) - if got != string(EncodeKey(tc.want)) { + got := string(EngineComparer.Successor(nil, EncodeMVCCKey(tc.a))) + if got != string(EncodeMVCCKey(tc.want)) { t.Errorf("a = %q: got %q, want %q", tc.a, got, tc.want) } }) @@ -610,7 +610,7 @@ func BenchmarkMVCCKeyCompare(b *testing.B) { WallTime: int64(rng.Intn(5)), }, } - keys[i] = EncodeKey(k) + keys[i] = EncodeMVCCKey(k) } b.ResetTimer() @@ -1111,15 +1111,15 @@ func TestPebbleMVCCTimeIntervalCollector(t *testing.T) { // The added key was not an MVCCKey. finishAndCheck(0, 0) require.NoError(t, collector.Add(pebble.InternalKey{ - UserKey: EncodeKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 2, Logical: 1}})}, + UserKey: EncodeMVCCKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 2, Logical: 1}})}, []byte("foo"))) // Added 1 MVCCKey which sets both the upper and lower bound. finishAndCheck(2, 3) require.NoError(t, collector.Add(pebble.InternalKey{ - UserKey: EncodeKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 22, Logical: 1}})}, + UserKey: EncodeMVCCKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 22, Logical: 1}})}, []byte("foo"))) require.NoError(t, collector.Add(pebble.InternalKey{ - UserKey: EncodeKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 25, Logical: 1}})}, + UserKey: EncodeMVCCKey(MVCCKey{aKey, hlc.Timestamp{WallTime: 25, Logical: 1}})}, []byte("foo"))) // Added 2 MVCCKeys. finishAndCheck(22, 26) diff --git a/pkg/storage/sst_iterator.go b/pkg/storage/sst_iterator.go index d3e52851e8ce..9bd8b49b1b39 100644 --- a/pkg/storage/sst_iterator.go +++ b/pkg/storage/sst_iterator.go @@ -88,7 +88,7 @@ func (r *sstIterator) SeekGE(key MVCCKey) { return } } - r.keyBuf = EncodeKeyToBuf(r.keyBuf, key) + r.keyBuf = EncodeMVCCKeyToBuf(r.keyBuf, key) var iKey *sstable.InternalKey trySeekUsingNext := false if r.seekGELastOp { diff --git a/pkg/storage/sst_writer.go b/pkg/storage/sst_writer.go index 8080d06f12da..a50a5f39aee9 100644 --- a/pkg/storage/sst_writer.go +++ b/pkg/storage/sst_writer.go @@ -120,8 +120,8 @@ func (fw *SSTWriter) clearRange(start, end MVCCKey) error { return errors.New("cannot call ClearRange on a closed writer") } fw.DataSize += int64(len(start.Key)) + int64(len(end.Key)) - fw.scratch = EncodeKeyToBuf(fw.scratch[:0], start) - return fw.fw.DeleteRange(fw.scratch, EncodeKey(end)) + fw.scratch = EncodeMVCCKeyToBuf(fw.scratch[:0], start) + return fw.fw.DeleteRange(fw.scratch, EncodeMVCCKey(end)) } // Put puts a kv entry into the sstable being built. An error is returned if it @@ -135,7 +135,7 @@ func (fw *SSTWriter) Put(key MVCCKey, value []byte) error { return errors.New("cannot call Put on a closed writer") } fw.DataSize += int64(len(key.Key)) + int64(len(value)) - fw.scratch = EncodeKeyToBuf(fw.scratch[:0], key) + fw.scratch = EncodeMVCCKeyToBuf(fw.scratch[:0], key) return fw.fw.Set(fw.scratch, value) } @@ -189,7 +189,7 @@ func (fw *SSTWriter) put(key MVCCKey, value []byte) error { return errors.New("cannot call Put on a closed writer") } fw.DataSize += int64(len(key.Key)) + int64(len(value)) - fw.scratch = EncodeKeyToBuf(fw.scratch[:0], key) + fw.scratch = EncodeMVCCKeyToBuf(fw.scratch[:0], key) return fw.fw.Set(fw.scratch, value) } @@ -247,7 +247,7 @@ func (fw *SSTWriter) clear(key MVCCKey) error { if fw.fw == nil { return errors.New("cannot call Clear on a closed writer") } - fw.scratch = EncodeKeyToBuf(fw.scratch[:0], key) + fw.scratch = EncodeMVCCKeyToBuf(fw.scratch[:0], key) fw.DataSize += int64(len(key.Key)) return fw.fw.Delete(fw.scratch) } @@ -268,7 +268,7 @@ func (fw *SSTWriter) Merge(key MVCCKey, value []byte) error { return errors.New("cannot call Merge on a closed writer") } fw.DataSize += int64(len(key.Key)) + int64(len(value)) - fw.scratch = EncodeKeyToBuf(fw.scratch[:0], key) + fw.scratch = EncodeMVCCKeyToBuf(fw.scratch[:0], key) return fw.fw.Merge(fw.scratch, value) }