Skip to content

Commit

Permalink
Merge #109091
Browse files Browse the repository at this point in the history
109091: storage: use `uuid.UUID` for `LockTableKey.TxnUUID` r=nvanbenschoten a=nvanbenschoten

This commit changes `LockTableKey.TxnUUID` from a `[]byte` to a `uuid.UUID`.

The field was a byte slice "to avoid copying a slice when decoding". This reasoning did not make much sense though, because the uuid slice was only 16 bytes in size. A slice header is 24 bytes, so using a uuid results in less copying. This is all presumably negligible though, so we make this change to make LockTableKey easier to work with.

Epic: None
Release note: None

Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
craig[bot] and nvanbenschoten committed Aug 21, 2023
2 parents 7970632 + 9adda04 commit ceccd41
Show file tree
Hide file tree
Showing 9 changed files with 31 additions and 36 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/rditer/replica_data_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
},
}

Expand Down
11 changes: 3 additions & 8 deletions pkg/storage/engine_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down Expand Up @@ -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")
}
Expand All @@ -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
}

Expand Down
12 changes: 6 additions & 6 deletions pkg/storage/engine_key_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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,
},
},

Expand Down Expand Up @@ -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
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/intent_interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/intent_interleaving_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down Expand Up @@ -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{
Expand Down Expand Up @@ -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))
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/intent_reader_writer.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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)
}
Expand Down
22 changes: 11 additions & 11 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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:
Expand Down Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions pkg/storage/pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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)
Expand Down

0 comments on commit ceccd41

Please sign in to comment.