Skip to content

Commit

Permalink
Merge #140122
Browse files Browse the repository at this point in the history
140122: storage: pass TxnMeta to MVCCAcquireLock r=arulajmani a=stevendanna

This passes the TxnMeta and IgnoredSeqNums as separate arguments. This is in advance of a potential new user of the API and helps ensure we don't add more dependencies on data not in the TxnMeta.

Informs #139141

Release note: None

Co-authored-by: Steven Danna <[email protected]>
  • Loading branch information
craig[bot] and stevendanna committed Jan 31, 2025
2 parents cc3365f + ee391b7 commit a81e1b1
Show file tree
Hide file tree
Showing 14 changed files with 53 additions and 40 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ func TestQueryResolvedTimestamp(t *testing.T) {
}
writeLock := func(k string, str lock.Strength) {
txn := roachpb.MakeTransaction("test", roachpb.Key(k), 0, 0, makeTS(1), 0, 1, 0, false /* omitInRangefeeds */)
err := storage.MVCCAcquireLock(ctx, db, &txn, str, roachpb.Key(k), nil, 0, 0)
err := storage.MVCCAcquireLock(ctx, db, &txn.TxnMeta, txn.IgnoredSeqNums, str, roachpb.Key(k), nil, 0, 0)
require.NoError(t, err)
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/lock.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,7 @@ func acquireLockOnKey(
// conflicts with un-contended replicated locks -- we need to do so before
// we can acquire our own replicated lock; do that now, and also acquire
// the replicated lock if no conflicts are found.
if err := storage.MVCCAcquireLock(ctx, readWriter, txn, str, key, ms, maxLockConflicts, targetLockConflictBytes); err != nil {
if err := storage.MVCCAcquireLock(ctx, readWriter, &txn.TxnMeta, txn.IgnoredSeqNums, str, key, ms, maxLockConflicts, targetLockConflictBytes); err != nil {
return roachpb.LockAcquisition{}, err
}
default:
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/lock_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,9 +195,9 @@ func TestTxnBoundReplicatedLockTableView(t *testing.T) {
txn2 := roachpb.MakeTransaction("txn2", keyA, isolation.Serializable, roachpb.NormalUserPriority, makeTS(100, 0), 0, 0, 0, false)

// Have txn1 acquire 2 locks with different strengths.
err = storage.MVCCAcquireLock(ctx, engine, &txn1, lock.Exclusive, keyA, nil, 0, 0)
err = storage.MVCCAcquireLock(ctx, engine, &txn1.TxnMeta, txn1.IgnoredSeqNums, lock.Exclusive, keyA, nil, 0, 0)
require.NoError(t, err)
err = storage.MVCCAcquireLock(ctx, engine, &txn1, lock.Shared, keyB, nil, 0, 0)
err = storage.MVCCAcquireLock(ctx, engine, &txn1.TxnMeta, txn1.IgnoredSeqNums, lock.Shared, keyB, nil, 0, 0)
require.NoError(t, err)

reader := engine.NewReader(storage.StandardDurability)
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/gc/gc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,9 +148,9 @@ func TestLockAgeThresholdSetting(t *testing.T) {
require.NoError(t, err)
// Acquire some shared and exclusive locks as well.
for _, txn := range []*roachpb.Transaction{&txn1, &txn2} {
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, txn, lock.Shared, makeKey(local, lock.Shared), nil, 0, 0))
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn.TxnMeta, txn.IgnoredSeqNums, lock.Shared, makeKey(local, lock.Shared), nil, 0, 0))
}
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn1, lock.Exclusive, makeKey(local, lock.Exclusive), nil, 0, 0))
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn1.TxnMeta, txn1.IgnoredSeqNums, lock.Exclusive, makeKey(local, lock.Exclusive), nil, 0, 0))
}
require.NoError(t, eng.Flush())

Expand Down Expand Up @@ -217,9 +217,9 @@ func TestIntentCleanupBatching(t *testing.T) {
idx := i*len(objectKeys) + j
switch idx % 3 {
case 0:
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn, lock.Shared, key, nil, 0, 0))
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn.TxnMeta, txn.IgnoredSeqNums, lock.Shared, key, nil, 0, 0))
case 1:
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn, lock.Exclusive, key, nil, 0, 0))
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn.TxnMeta, txn.IgnoredSeqNums, lock.Exclusive, key, nil, 0, 0))
case 2:
_, err := storage.MVCCPut(ctx, eng, key, intentHlc, value, storage.MVCCWriteOptions{Txn: &txn})
require.NoError(t, err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/rangefeed/task_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -311,7 +311,7 @@ func TestInitResolvedTSScan(t *testing.T) {
roachpb.MakeLock(&txn1.TxnMeta, roachpb.Key("p"), lock.Exclusive),
}
for _, l := range testLocks {
err := storage.MVCCAcquireLock(ctx, engine, &txn1, l.Strength, l.Key, nil, 0, 0)
err := storage.MVCCAcquireLock(ctx, engine, &txn1.TxnMeta, txn1.IgnoredSeqNums, l.Strength, l.Key, nil, 0, 0)
require.NoError(t, err)
}
return engine
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/replica_consistency_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -328,7 +328,7 @@ func TestReplicaChecksumSHA512(t *testing.T) {
for i, l := range locks {
txnID := uuid.FromUint128(uint128.FromInts(0, uint64(l.txnID)))
txn := &roachpb.Transaction{TxnMeta: enginepb.TxnMeta{ID: txnID}}
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, txn, l.str, roachpb.Key(l.key), nil, 0, 0))
require.NoError(t, storage.MVCCAcquireLock(ctx, eng, &txn.TxnMeta, txn.IgnoredSeqNums, l.str, roachpb.Key(l.key), nil, 0, 0))

rd, err = CalcReplicaDigest(ctx, desc, eng, kvpb.ChecksumMode_CHECK_FULL, unlim, nil /* settings */)
require.NoError(t, err)
Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2004,11 +2004,11 @@ func runMVCCAcquireLockCommon(
txn = &txn2
}
// Acquire a shared and an exclusive lock on the key.
err := MVCCAcquireLock(ctx, eng, txn, lock.Shared, key, nil, 0, 0)
err := MVCCAcquireLock(ctx, eng, &txn.TxnMeta, txn.IgnoredSeqNums, lock.Shared, key, nil, 0, 0)
if err != nil {
b.Fatal(err)
}
err = MVCCAcquireLock(ctx, eng, txn, lock.Exclusive, key, nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn.TxnMeta, txn.IgnoredSeqNums, lock.Exclusive, key, nil, 0, 0)
if err != nil {
b.Fatal(err)
}
Expand All @@ -2032,7 +2032,7 @@ func runMVCCAcquireLockCommon(
if checkFor {
err = MVCCCheckForAcquireLock(ctx, rw, txn, strength, key, 0, 0)
} else {
err = MVCCAcquireLock(ctx, rw, txn, strength, key, ms, 0, 0)
err = MVCCAcquireLock(ctx, rw, &txn.TxnMeta, txn.IgnoredSeqNums, strength, key, ms, 0, 0)
}
if heldOtherTxn {
if err == nil {
Expand Down
36 changes: 18 additions & 18 deletions pkg/storage/engine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1533,9 +1533,9 @@ func TestGetIntent(t *testing.T) {
// Key "b" has an intent, an exclusive lock, and a shared lock from txn1.
// NOTE: acquire in increasing strength order so that acquisition is never
// skipped.
err = MVCCAcquireLock(ctx, eng, txn1, lock.Shared, keyB, nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn1.TxnMeta, txn1.IgnoredSeqNums, lock.Shared, keyB, nil, 0, 0)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, eng, txn1, lock.Exclusive, keyB, nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn1.TxnMeta, txn1.IgnoredSeqNums, lock.Exclusive, keyB, nil, 0, 0)
require.NoError(t, err)
_, err = MVCCPut(ctx, eng, keyB, txn1.ReadTimestamp, val, MVCCWriteOptions{Txn: txn1})
require.NoError(t, err)
Expand All @@ -1545,15 +1545,15 @@ func TestGetIntent(t *testing.T) {
require.NoError(t, err)

// Key "d" has an exclusive lock and a shared lock from txn2.
err = MVCCAcquireLock(ctx, eng, txn2, lock.Shared, keyD, nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn2.TxnMeta, txn2.IgnoredSeqNums, lock.Shared, keyD, nil, 0, 0)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, eng, txn2, lock.Exclusive, keyD, nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn2.TxnMeta, txn2.IgnoredSeqNums, lock.Exclusive, keyD, nil, 0, 0)
require.NoError(t, err)

// Key "e" has a shared lock from each txn.
err = MVCCAcquireLock(ctx, eng, txn1, lock.Shared, keyE, nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn1.TxnMeta, txn1.IgnoredSeqNums, lock.Shared, keyE, nil, 0, 0)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, eng, txn2, lock.Shared, keyE, nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn2.TxnMeta, txn2.IgnoredSeqNums, lock.Shared, keyE, nil, 0, 0)
require.NoError(t, err)

// Key "f" has no intent/locks.
Expand Down Expand Up @@ -1641,7 +1641,7 @@ func TestScanLocks(t *testing.T) {
if str == lock.Intent {
_, err = MVCCPut(ctx, eng, roachpb.Key(k), txn1.ReadTimestamp, roachpb.Value{RawBytes: roachpb.Key(k)}, MVCCWriteOptions{Txn: txn1})
} else {
err = MVCCAcquireLock(ctx, eng, txn1, str, roachpb.Key(k), nil, 0, 0)
err = MVCCAcquireLock(ctx, eng, &txn1.TxnMeta, txn1.IgnoredSeqNums, str, roachpb.Key(k), nil, 0, 0)
}
require.NoError(t, err)
}
Expand Down Expand Up @@ -2234,11 +2234,11 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
txnB := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err := MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnB, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err = MVCCAcquireLock(ctx, rw, &txnB.TxnMeta, txnB.IgnoredSeqNums, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err = MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
},
start: keyA,
Expand All @@ -2252,9 +2252,9 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
name: "shared and exclusive locks should be ignored no end key",
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err := MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err = MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Exclusive, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
},
start: keyA,
Expand All @@ -2267,11 +2267,11 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
setup: func(t *testing.T, rw ReadWriter, _ *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
txnB := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err := MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnB, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err = MVCCAcquireLock(ctx, rw, &txnB.TxnMeta, txnB.IgnoredSeqNums, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err = MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
require.NoError(t, err)
_, err = MVCCPut(ctx, rw, keyC, txnA.WriteTimestamp, val, MVCCWriteOptions{Txn: txnA})
Expand All @@ -2287,11 +2287,11 @@ func TestScanConflictingIntentsForDroppingLatchesEarly(t *testing.T) {
setup: func(t *testing.T, rw ReadWriter, txn *roachpb.Transaction) {
txnA := newTxn(belowTxnTS)
txnB := newTxn(belowTxnTS)
err := MVCCAcquireLock(ctx, rw, txnA, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err := MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnB, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err = MVCCAcquireLock(ctx, rw, &txnB.TxnMeta, txnB.IgnoredSeqNums, lock.Shared, keyA, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
err = MVCCAcquireLock(ctx, rw, txnA, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
err = MVCCAcquireLock(ctx, rw, &txnA.TxnMeta, txnA.IgnoredSeqNums, lock.Exclusive, keyB, nil /*ms*/, 0 /*maxConflicts*/, 0 /*targetLockConflictBytes*/)
require.NoError(t, err)
_, err = MVCCPut(ctx, rw, keyC, txn.WriteTimestamp, val, MVCCWriteOptions{Txn: txn})
require.NoError(t, err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/metamorphic/operations.go
Original file line number Diff line number Diff line change
Expand Up @@ -316,7 +316,7 @@ func (m mvccAcquireLockOp) run(ctx context.Context) string {
txn := m.m.getTxn(m.txn)
writer := m.m.getReadWriter(m.writer)

err := storage.MVCCAcquireLock(ctx, writer, txn, m.strength, m.key, nil, int64(m.maxLockConflicts), m.targetLockConflictBytes)
err := storage.MVCCAcquireLock(ctx, writer, &txn.TxnMeta, txn.IgnoredSeqNums, m.strength, m.key, nil, int64(m.maxLockConflicts), m.targetLockConflictBytes)
if err != nil {
return fmt.Sprintf("error: %s", err)
}
Expand Down
9 changes: 5 additions & 4 deletions pkg/storage/mvcc.go
Original file line number Diff line number Diff line change
Expand Up @@ -6063,7 +6063,8 @@ func MVCCCheckForAcquireLock(
func MVCCAcquireLock(
ctx context.Context,
rw ReadWriter,
txn *roachpb.Transaction,
txn *enginepb.TxnMeta,
ignoredSeqNums []enginepb.IgnoredSeqNumRange,
str lock.Strength,
key roachpb.Key,
ms *enginepb.MVCCStats,
Expand Down Expand Up @@ -6135,7 +6136,7 @@ func MVCCAcquireLock(
"cannot acquire lock with strength %s at seq number %d, "+
"already held at higher seq number %d",
str.String(), txn.Sequence, foundLock.Txn.Sequence)
} else if enginepb.TxnSeqIsIgnored(foundLock.Txn.Sequence, txn.IgnoredSeqNums) {
} else if enginepb.TxnSeqIsIgnored(foundLock.Txn.Sequence, ignoredSeqNums) {
// Acquiring at same epoch and new sequence number after
// previous sequence number was rolled back.
//
Expand All @@ -6161,7 +6162,7 @@ func MVCCAcquireLock(
// can still avoid reacquisition.
inHistoryNotRolledBack := false
for _, e := range foundLock.IntentHistory {
if !enginepb.TxnSeqIsIgnored(e.Sequence, txn.IgnoredSeqNums) {
if !enginepb.TxnSeqIsIgnored(e.Sequence, ignoredSeqNums) {
inHistoryNotRolledBack = true
break
}
Expand Down Expand Up @@ -6194,7 +6195,7 @@ func MVCCAcquireLock(
defer buf.release()

newMeta := &buf.newMeta
newMeta.Txn = &txn.TxnMeta
newMeta.Txn = txn
newMeta.Timestamp = txn.WriteTimestamp.ToLegacyTimestamp()
keyBytes, valBytes, err := buf.putLockMeta(rw, key, str, newMeta, rolledBack)
if err != nil {
Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/mvcc_history_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1183,7 +1183,13 @@ func cmdAcquireLock(e *evalCtx) error {
str := e.getStrength()
maxLockConflicts := e.getMaxLockConflicts()
targetLockConflictBytes := e.getTargetLockConflictBytes()
return storage.MVCCAcquireLock(e.ctx, rw, txn, str, key, e.ms, maxLockConflicts, targetLockConflictBytes)
var txnMeta *enginepb.TxnMeta
var ignoredSeq []enginepb.IgnoredSeqNumRange
if txn != nil {
txnMeta = &txn.TxnMeta
ignoredSeq = txn.IgnoredSeqNums
}
return storage.MVCCAcquireLock(e.ctx, rw, txnMeta, ignoredSeq, str, key, e.ms, maxLockConflicts, targetLockConflictBytes)
})
}

Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/mvcc_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2055,7 +2055,13 @@ func TestMVCCStatsRandomized(t *testing.T) {
if s.rng.Intn(2) != 0 {
str = lock.Exclusive
}
if err := MVCCAcquireLock(ctx, s.batch, s.Txn, str, s.key, s.MSDelta, 0, 0); err != nil {
var txnMeta *enginepb.TxnMeta
var ignoredSeq []enginepb.IgnoredSeqNumRange
if s.Txn != nil {
txnMeta = &s.Txn.TxnMeta
ignoredSeq = s.Txn.IgnoredSeqNums
}
if err := MVCCAcquireLock(ctx, s.batch, txnMeta, ignoredSeq, str, s.key, s.MSDelta, 0, 0); err != nil {
return false, err.Error()
}
return true, ""
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/mvcc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2206,7 +2206,7 @@ func TestMVCCClearTimeRange(t *testing.T) {

// Add a shared lock at k1 with a txn at ts3.
addLock := func(t *testing.T, rw ReadWriter) {
err := MVCCAcquireLock(ctx, rw, &txn, lock.Shared, testKey1, nil, 0, 0)
err := MVCCAcquireLock(ctx, rw, &txn.TxnMeta, txn.IgnoredSeqNums, lock.Shared, testKey1, nil, 0, 0)
require.NoError(t, err)
}
t.Run("clear everything hitting lock fails", func(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/storage/sst_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,7 @@ func TestCheckSSTConflictsMaxLockConflicts(t *testing.T) {
if i%2 != 0 {
str = lock.Exclusive
}
require.NoError(t, MVCCAcquireLock(ctx, batch, txn1, str, roachpb.Key(key), nil, 0, 0))
require.NoError(t, MVCCAcquireLock(ctx, batch, &txn1.TxnMeta, txn1.IgnoredSeqNums, str, roachpb.Key(key), nil, 0, 0))
}
require.NoError(t, batch.Commit(true))
batch.Close()
Expand Down

0 comments on commit a81e1b1

Please sign in to comment.