diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go index 5ae795d04aae..215082997776 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp_test.go @@ -229,7 +229,7 @@ func TestQueryResolvedTimestampErrors(t *testing.T) { lockTableKey := storage.LockTableKey{ Key: roachpb.Key("a"), Strength: lock.Exclusive, - TxnUUID: txnUUID.GetBytes(), + TxnUUID: txnUUID, } engineKey, buf := lockTableKey.ToEngineKey(nil) diff --git a/pkg/kv/kvserver/rditer/replica_data_iter_test.go b/pkg/kv/kvserver/rditer/replica_data_iter_test.go index 87a4964fe76a..152fe709c47a 100644 --- a/pkg/kv/kvserver/rditer/replica_data_iter_test.go +++ b/pkg/kv/kvserver/rditer/replica_data_iter_test.go @@ -111,11 +111,11 @@ func createRangeData( { Key: keys.RangeDescriptorKey(desc.StartKey), // mark [1] above as intent Strength: lock.Exclusive, - TxnUUID: testTxnID.GetBytes(), + TxnUUID: testTxnID, }, { Key: desc.StartKey.AsRawKey(), // mark [2] above as intent Strength: lock.Exclusive, - TxnUUID: testTxnID.GetBytes(), + TxnUUID: testTxnID, }, } diff --git a/pkg/storage/engine_key.go b/pkg/storage/engine_key.go index b8505df00999..c0dc3de8503f 100644 --- a/pkg/storage/engine_key.go +++ b/pkg/storage/engine_key.go @@ -177,7 +177,7 @@ func (k EngineKey) ToLockTableKey() (LockTableKey, error) { if key.Strength < lock.None || key.Strength > lock.Exclusive { return LockTableKey{}, errors.Errorf("unknown strength %d", key.Strength) } - key.TxnUUID = k.Version[1:] + key.TxnUUID = *(*uuid.UUID)(k.Version[1:]) default: return LockTableKey{}, errors.Errorf("version is not valid for a LockTableKey %x", k.Version) } @@ -252,18 +252,13 @@ func (m EngineKeyFormatter) Format(f fmt.State, c rune) { type LockTableKey struct { Key roachpb.Key Strength lock.Strength - // Slice is of length uuid.Size. We use a slice instead of a byte array, to - // avoid copying a slice when decoding. - TxnUUID []byte + TxnUUID uuid.UUID } // ToEngineKey converts a lock table key to an EngineKey. buf is used as // scratch-space to avoid allocations -- its contents will be overwritten and // not appended to. func (lk LockTableKey) ToEngineKey(buf []byte) (EngineKey, []byte) { - if len(lk.TxnUUID) != uuid.Size { - panic("invalid TxnUUID") - } if lk.Strength != lock.Exclusive { panic("unsupported lock strength") } @@ -283,7 +278,7 @@ func (lk LockTableKey) ToEngineKey(buf []byte) (EngineKey, []byte) { k.Version = make([]byte, engineKeyVersionLockTableLen) } k.Version[0] = byte(lk.Strength) - copy(k.Version[1:], lk.TxnUUID) + copy(k.Version[1:], lk.TxnUUID[:]) return k, buf } diff --git a/pkg/storage/engine_key_test.go b/pkg/storage/engine_key_test.go index 2520c4861dd3..e2abe9b79751 100644 --- a/pkg/storage/engine_key_test.go +++ b/pkg/storage/engine_key_test.go @@ -36,13 +36,13 @@ func TestLockTableKeyEncodeDecode(t *testing.T) { testCases := []struct { key LockTableKey }{ - {key: LockTableKey{Key: roachpb.Key("foo"), Strength: lock.Exclusive, TxnUUID: uuid1[:]}}, - {key: LockTableKey{Key: roachpb.Key("a"), Strength: lock.Exclusive, TxnUUID: uuid2[:]}}, + {key: LockTableKey{Key: roachpb.Key("foo"), Strength: lock.Exclusive, TxnUUID: uuid1}}, + {key: LockTableKey{Key: roachpb.Key("a"), Strength: lock.Exclusive, TxnUUID: uuid2}}, // Causes a doubly-local range local key. {key: LockTableKey{ Key: keys.RangeDescriptorKey(roachpb.RKey("baz")), Strength: lock.Exclusive, - TxnUUID: uuid1[:]}}, + TxnUUID: uuid1}}, } buf := make([]byte, 100) for i, test := range testCases { @@ -177,14 +177,14 @@ func TestEngineKeyValidate(t *testing.T) { key: LockTableKey{ Key: roachpb.Key("foo"), Strength: lock.Exclusive, - TxnUUID: uuid1[:], + TxnUUID: uuid1, }, }, { key: LockTableKey{ Key: keys.RangeDescriptorKey(roachpb.RKey("bar")), Strength: lock.Exclusive, - TxnUUID: uuid1[:], + TxnUUID: uuid1, }, }, @@ -272,7 +272,7 @@ func randomLockTableKey(r *rand.Rand) LockTableKey { } var txnID uuid.UUID txnID.DeterministicV4(r.Uint64(), math.MaxUint64) - k.TxnUUID = txnID[:] + k.TxnUUID = txnID return k } diff --git a/pkg/storage/intent_interleaving_iter.go b/pkg/storage/intent_interleaving_iter.go index 7139fc35ab7e..ab105c8c4e74 100644 --- a/pkg/storage/intent_interleaving_iter.go +++ b/pkg/storage/intent_interleaving_iter.go @@ -551,7 +551,7 @@ func (i *intentInterleavingIter) SeekIntentGE(key roachpb.Key, txnUUID uuid.UUID engineKey, i.intentKeyBuf = LockTableKey{ Key: key, Strength: lock.Exclusive, - TxnUUID: txnUUID[:], + TxnUUID: txnUUID, }.ToEngineKey(i.intentKeyBuf) var limitKey roachpb.Key if i.iterValid && !i.prefix { diff --git a/pkg/storage/intent_interleaving_iter_test.go b/pkg/storage/intent_interleaving_iter_test.go index 5feea36beb47..0f22e2fd141f 100644 --- a/pkg/storage/intent_interleaving_iter_test.go +++ b/pkg/storage/intent_interleaving_iter_test.go @@ -294,7 +294,7 @@ func TestIntentInterleavingIter(t *testing.T) { return err.Error() } } else { - ltKey := LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID[:]} + ltKey := LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID} eKey, _ := ltKey.ToEngineKey(nil) if err := batch.PutEngineKey(eKey, val); err != nil { return err.Error() @@ -561,7 +561,7 @@ func generateRandomData( } val, err := protoutil.Marshal(&meta) require.NoError(t, err) - ltKey := LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID[:]} + ltKey := LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID} lkv = append(lkv, lockKeyValue{ key: ltKey, val: val, liveIntent: hasIntent && i == 0}) mvcckv = append(mvcckv, MVCCKeyValue{ @@ -823,7 +823,7 @@ func writeBenchData( require.NoError(b, err) if separated { eKey, _ := - LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID[:]}.ToEngineKey(nil) + LockTableKey{Key: key, Strength: lock.Exclusive, TxnUUID: txnUUID}.ToEngineKey(nil) require.NoError(b, batch.PutEngineKey(eKey, val)) } else { require.NoError(b, batch.PutUnversioned(key, val)) diff --git a/pkg/storage/intent_reader_writer.go b/pkg/storage/intent_reader_writer.go index 92c8d0670920..1acf6cefcdcf 100644 --- a/pkg/storage/intent_reader_writer.go +++ b/pkg/storage/intent_reader_writer.go @@ -43,7 +43,7 @@ func (idw intentDemuxWriter) ClearIntent( engineKey, buf = LockTableKey{ Key: key, Strength: lock.Exclusive, - TxnUUID: txnUUID[:], + TxnUUID: txnUUID, }.ToEngineKey(buf) if txnDidNotUpdateMeta { return buf, idw.w.SingleClearEngineKey(engineKey) @@ -61,7 +61,7 @@ func (idw intentDemuxWriter) PutIntent( engineKey, buf = LockTableKey{ Key: key, Strength: lock.Exclusive, - TxnUUID: txnUUID[:], + TxnUUID: txnUUID, }.ToEngineKey(buf) return buf, idw.w.PutEngineKey(engineKey, value) } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index 1e6daec6a4f1..a6c2857ab8f0 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -6455,6 +6455,16 @@ func computeStatsForIterWithVisitors( return ms, nil } +// MVCCIsSpanEmptyOptions configures the MVCCIsSpanEmpty function. +type MVCCIsSpanEmptyOptions struct { + // StartKey determines start of the checked span. + StartKey roachpb.Key + // EndKey determines the end of exported interval (exclusive). + EndKey roachpb.Key + // StartTS and EndTS determine the scanned time range as (startTS, endTS]. + StartTS, EndTS hlc.Timestamp +} + // MVCCIsSpanEmpty returns true if there are no MVCC keys whatsoever in the key // span in the requested time interval. If a time interval is given and any // inline values are encountered, an error may be returned. @@ -7035,16 +7045,6 @@ type MVCCExportFingerprintOptions struct { StripIndexPrefixAndTimestamp bool } -// MVCCIsSpanEmptyOptions configures the MVCCIsSpanEmpty function. -type MVCCIsSpanEmptyOptions struct { - // StartKey determines start of the checked span. - StartKey roachpb.Key - // EndKey determines the end of exported interval (exclusive). - EndKey roachpb.Key - // StartTS and EndTS determine the scanned time range as (startTS, endTS]. - StartTS, EndTS hlc.Timestamp -} - // PeekRangeKeysLeft peeks for any range keys to the left of the given key. // It returns the relative position of any range keys to the peek key, along // with the (unsafe) range key stack: @@ -7245,7 +7245,7 @@ func ReplacePointTombstonesWithRangeTombstones( // In order to test the correctness of range deletion tombstones, we added a // testing knob to replace point deletions with range deletion tombstones in -// some tests. Unfortuantely, doing so affects the correctness of rangefeeds. +// some tests. Unfortunately, doing so affects the correctness of rangefeeds. // The tests in question do not use rangefeeds, but some system functionality // does use rangefeeds internally. The primary impact is that catch-up scans // will miss deletes. That makes these issues rare and hard to detect. In order diff --git a/pkg/storage/pebble_test.go b/pkg/storage/pebble_test.go index 201ba2f24bd4..ebe712902eb5 100644 --- a/pkg/storage/pebble_test.go +++ b/pkg/storage/pebble_test.go @@ -601,7 +601,7 @@ func TestPebbleMVCCTimeIntervalCollector(t *testing.T) { // Nothing added. finishAndCheck(0, 0) uuid := uuid.Must(uuid.FromString("6ba7b810-9dad-11d1-80b4-00c04fd430c8")) - ek, _ := LockTableKey{aKey, lock.Exclusive, uuid[:]}.ToEngineKey(nil) + ek, _ := LockTableKey{aKey, lock.Exclusive, uuid}.ToEngineKey(nil) require.NoError(t, collector.Add(pebble.InternalKey{UserKey: ek.Encode()}, []byte("foo"))) // The added key was not an MVCCKey. finishAndCheck(0, 0) @@ -1241,7 +1241,7 @@ func TestShortAttributeExtractor(t *testing.T) { var txnUUID [uuid.Size]byte lockKey, _ := LockTableKey{ - Key: roachpb.Key("a"), Strength: lock.Exclusive, TxnUUID: txnUUID[:]}.ToEngineKey(nil) + Key: roachpb.Key("a"), Strength: lock.Exclusive, TxnUUID: txnUUID}.ToEngineKey(nil) v := MVCCValue{} tombstoneVal, err := EncodeMVCCValue(v) require.NoError(t, err)