From 023f676e1a96ba589bb35225763ead44bdb7cae8 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Thu, 19 Oct 2023 17:34:22 -0500 Subject: [PATCH] concurrency: correctly handle lock updates that decrease its strength When a lock is updated on behalf of a non-finalized transaction, the lock table simply forgets any replicated lock information it was previously tracking. Now, if a lock is held with both replicated and unreplicated durability, and the replicated lock's strength is stronger than the unreplicated lock's strength, forgetting the replicated lock means there may be requests waiting on a key that no longer conflict with what's being tracked in memory[1]. In such cases, requests that no longer conflict with the in-memory tracking should no longer wait on this key -- they should be allowed to proceed with their scan. It's not guaranteed that they no longer conflict with the lock -- they might rediscover the replicated lock again and start waiting. However, if they no longer conflict with the lock, blocking them indefinitely can lead to undetected deadlocks in rare cases[2]. [1] Concretely, consider the following construction: - key a: [txn 1, (repl intent ts@10), (unrepl shared)] - wait-queue readers: [none ts@12], [none ts@15] - wait-queue locking requests: [shared], [shared], [exclusive] [shared] In this case, updating the intent's timestamp to ts@14 (and forgetting it) should allow both the non-locking readers and first two shared locking requests to become compatible with what's being tracked in memory for this key. [2] This can happen if the lock update corresponds to a successful PUSH_TIMESTAMP. The pusher will be blocked in the lock table waiter waiting for a `waitingState` update that never comes. Now, if there was a dependency cycle between the pusher and pushee had the push timestamp been blocking, we would find ourselves in an undetected deeadlock situation. Fixes #112608 Release note: None --- pkg/kv/kvserver/concurrency/lock/locking.go | 9 + .../kvserver/concurrency/lock/locking_test.go | 65 ++++++ pkg/kv/kvserver/concurrency/lock_table.go | 107 ++++++++-- .../testdata/concurrency_manager/shared_locks | 195 ++++++++++++++++++ .../testdata/lock_table/shared_locks | 105 ++++++++++ .../concurrency/testdata/lock_table/update | 72 +++++++ 6 files changed, 541 insertions(+), 12 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/lock/locking.go b/pkg/kv/kvserver/concurrency/lock/locking.go index 8554b7ac89e8..b3926411ad0c 100644 --- a/pkg/kv/kvserver/concurrency/lock/locking.go +++ b/pkg/kv/kvserver/concurrency/lock/locking.go @@ -127,6 +127,15 @@ func (m *Mode) Empty() bool { return m.Strength == None && m.Timestamp.IsEmpty() } +// Less returns true if the receiver conflicts with fewer requests than the Mode +// supplied. +func (m Mode) Less(o Mode) bool { + if m.Strength == o.Strength { + return !m.Timestamp.Less(o.Timestamp) // lower timestamp conflicts with more requests + } + return m.Strength < o.Strength +} + // MakeModeNone constructs a Mode with strength None. func MakeModeNone(ts hlc.Timestamp, isoLevel isolation.Level) Mode { return Mode{ diff --git a/pkg/kv/kvserver/concurrency/lock/locking_test.go b/pkg/kv/kvserver/concurrency/lock/locking_test.go index a9c6fe708b30..5f82ae4d81ec 100644 --- a/pkg/kv/kvserver/concurrency/lock/locking_test.go +++ b/pkg/kv/kvserver/concurrency/lock/locking_test.go @@ -404,3 +404,68 @@ func TestCheckLockConflicts_IntentWithIntent(t *testing.T) { ) } } + +func TestLockModeLess(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + tsBelow := makeTS(1) + tsLock := makeTS(2) + tsAbove := makeTS(3) + + testCases := []struct { + m1 lock.Mode + m2 lock.Mode + exp bool + }{ + { + m1: lock.MakeModeNone(tsLock, isolation.Serializable), + m2: lock.MakeModeNone(tsBelow, isolation.Serializable), // stronger + exp: true, + }, + { + m1: lock.MakeModeNone(tsLock, isolation.Serializable), // stronger + m2: lock.MakeModeNone(tsAbove, isolation.Serializable), + exp: false, + }, + { + m1: lock.MakeModeIntent(tsLock), // stronger + m2: lock.MakeModeNone(tsBelow, isolation.Serializable), + exp: false, + }, + { + m1: lock.MakeModeIntent(tsLock), + m2: lock.MakeModeIntent(tsBelow), // stronger + exp: true, + }, + { + m1: lock.MakeModeIntent(tsLock), // stronger + m2: lock.MakeModeIntent(tsAbove), + exp: false, + }, + { + m1: lock.MakeModeIntent(tsLock), // stronger + m2: lock.MakeModeShared(), + exp: false, + }, + { + m1: lock.MakeModeIntent(tsLock), // stronger + m2: lock.MakeModeUpdate(), + exp: false, + }, + { + m1: lock.MakeModeIntent(tsLock), // stronger + m2: lock.MakeModeExclusive(tsBelow, isolation.Serializable), + exp: false, + }, + { + m1: lock.MakeModeIntent(tsLock), // stronger + m2: lock.MakeModeExclusive(tsAbove, isolation.Serializable), + exp: false, + }, + } + + for _, tc := range testCases { + require.Equal(t, tc.exp, tc.m1.Less(tc.m2)) + } +} diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 3b1295371604..b5a65c8d427b 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -2962,6 +2962,10 @@ func (kl *keyLocks) acquireLock( // here. if tl.getLockMode().Strength == lock.Exclusive || tl.getLockMode().Strength == lock.Intent { + // TODO(XXX): we can replace this with recomputeWaitQueues and get rid + // of increasedLockTS entirely. We also don't need to to be strict about + // only calling recomputeWaitQueues if the lock is held with Exclusive + // or Intent lock strength, but it doesn't hurt. kl.increasedLockTs(afterTs) } } @@ -3200,14 +3204,18 @@ func (kl *keyLocks) tryClearLock(force bool) bool { // transaction, else the lock is updated. Returns whether the keyLocks struct // can be garbage collected, and whether it was held by the txn. // Acquires l.mu. -func (kl *keyLocks) tryUpdateLock(up *roachpb.LockUpdate) (heldByTxn, gc bool) { +func (kl *keyLocks) tryUpdateLock( + up *roachpb.LockUpdate, st *cluster.Settings, +) (heldByTxn, gc bool) { kl.mu.Lock() defer kl.mu.Unlock() - return kl.tryUpdateLockLocked(*up) + return kl.tryUpdateLockLocked(*up, st) } // REQUIRES: kl.mu is locked. -func (kl *keyLocks) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bool) { +func (kl *keyLocks) tryUpdateLockLocked( + up roachpb.LockUpdate, st *cluster.Settings, +) (heldByTxn, gc bool) { if kl.isEmptyLock() { // Already free. This can happen when an unreplicated lock is removed in // tryActiveWait due to the txn being in the txnStatusCache. @@ -3242,6 +3250,7 @@ func (kl *keyLocks) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bo txn := &up.Txn ts := up.Txn.WriteTimestamp beforeTs := tl.writeTS() + beforeStr := tl.getLockMode().Strength advancedTs := beforeTs.Less(ts) isLocked := false // The MVCC keyspace is the source of truth about the disposition of a @@ -3321,14 +3330,10 @@ func (kl *keyLocks) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bo return true, gc } - if advancedTs { - // We only need to let through non-locking readers if the lock is held with - // strength {Exclusive,Intent}. See acquireLock for an explanation as to - // why. - if tl.getLockMode().Strength == lock.Exclusive || - tl.getLockMode().Strength == lock.Intent { - kl.increasedLockTs(ts) - } + afterStr := tl.getLockMode().Strength + + if beforeStr != afterStr || advancedTs { + kl.recomputeWaitQueues(st) } // Else no change for waiters. This can happen due to a race between different // callers of UpdateLocks(). @@ -3347,6 +3352,9 @@ func (kl *keyLocks) increasedLockTs(newTs hlc.Timestamp) { curr := e e = e.Next() if g.ts.Less(newTs) { + // TODO(arul): Flip this condition; as written, this short circuits -- + // once we've removed the distinguished waiter, we'll stop releasing + // readers that may no longer conflict with the lock. distinguishedRemoved = distinguishedRemoved || kl.removeReader(curr) } // Else don't inform an active waiter which continues to be an active waiter @@ -3357,6 +3365,81 @@ func (kl *keyLocks) increasedLockTs(newTs hlc.Timestamp) { } } +// recomputeWaitQueues goes through the receiver's wait queues and recomputes +// whether actively waiting requests should continue to do so, given the key's +// locks holders and other waiting requests. Such computation is necessary when +// a lock's strength has decreased[1] or locking requests have dropped out of +// wait queue's[2] without actually acquiring the lock. +// +// [1] This can happen as a result of savepoint rollback or when the lock table +// stops tracking a replicated lock because of a PUSH_TIMESTAMP that +// successfully bumps the pushee's timestamp. +// [2] A locking request that doesn't conflict with any held lock(s) may still +// have to actively wait if it conflicts with a lower sequence numbered request +// already in the lock's wait queue. Locking requests dropping out of a lock's +// wait queue can therefore result in other requests no longer needing to +// actively wait. +// +// TODO(arul): We could optimize this function if we had information about the +// context it was being called in. +// +// REQUIRES: kl.mu to be locked. +func (kl *keyLocks) recomputeWaitQueues(st *cluster.Settings) { + var strongestMode lock.Mode + for e := kl.holders.Front(); e != nil; e = e.Next() { + holder := e.Value + if strongestMode.Less(holder.getLockMode()) { + strongestMode = holder.getLockMode() + } + } + + distinguishedRemoved := false + for e := kl.waitingReaders.Front(); e != nil; { + reader := e.Value + curr := e + e = e.Next() + if !lock.Conflicts(reader.curLockMode(), strongestMode, &st.SV) { + distinguishedRemoved = kl.removeReader(curr) || distinguishedRemoved + } + } + for e := kl.queuedLockingRequests.Front(); e != nil; { + qlr := e.Value + curr := e + e = e.Next() + if lock.Conflicts(qlr.mode, strongestMode, &st.SV) { + break + } + if qlr.active { + // A queued locking request, that's actively waiting, no longer conflicts + // with locks on this key -- it can be allowed to proceed. There's two + // cases: + // 1. If it's a transactional request, it needs to acquire a claim by + // holding its place in the lock wait queue while marking itself as + // inactive. + // 2. Non-transactional requests do not acquire claims, so they can be + // removed from the wait queue. + if qlr.guard.txn == nil { + distinguishedRemoved = kl.removeLockingRequest(curr) || distinguishedRemoved + } else { + qlr.active = false // mark as inactive + if qlr.guard == kl.distinguishedWaiter { + kl.distinguishedWaiter = nil + distinguishedRemoved = true + } + qlr.guard.mu.Lock() + qlr.guard.doneActivelyWaitingAtLock() + qlr.guard.mu.Unlock() + } + } + if strongestMode.Less(qlr.mode) { + strongestMode = qlr.mode + } + } + if distinguishedRemoved { + kl.tryMakeNewDistinguished() + } +} + // removeLockingRequest removes the locking request (or non-transactional // writer), referenced by the supplied list.Element, from the lock's // queuedLockingRequests list. Returns whether the request was the distinguished @@ -4155,7 +4238,7 @@ func (t *lockTableImpl) updateLockInternal(up *roachpb.LockUpdate) (heldByTxn bo var locksToGC []*keyLocks heldByTxn = false changeFunc := func(l *keyLocks) { - held, gc := l.tryUpdateLock(up) + held, gc := l.tryUpdateLock(up, t.settings) heldByTxn = heldByTxn || held if gc { locksToGC = append(locksToGC, l) diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/shared_locks b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/shared_locks index b01c2bb5c272..022ec80c07b0 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/shared_locks +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/shared_locks @@ -13,6 +13,15 @@ new-txn name=txn4 ts=10,1 epoch=0 new-txn name=txn5 ts=10,1 epoch=0 ---- +new-txn name=txn6 ts=10,1 epoch=0 +---- + +new-txn name=txn7 ts=11,1 epoch=0 iso=read-committed +---- + +new-txn name=txn8 ts=10,1 epoch=0 +---- + # ----------------------------------------------------------------------------- # Ensure releasing the first of multiple shared lock holders results in correct # pushes. @@ -180,3 +189,189 @@ on-txn-updated txn=txn3 status=committed finish req=req5 ---- [-] finish req5: finishing request + +# ------------------------------------------------------------------------------ +# Ensure that when an intent is pushed out of the way by a non-locking read, but +# there is still a shared lock on the key, the non-locking read is able to +# proceed. Serves as a regression test for the bug identified in +# https://github.com/cockroachdb/cockroach/issues/112608; prior to the fix, the +# non-locking read could end up waiting indefinitely. +# ------------------------------------------------------------------------------ + +new-request name=req6 txn=txn6 ts=10,1 + put key=a value=v +---- + +sequence req=req6 +---- +[6] sequence req6: sequencing request +[6] sequence req6: acquiring latches +[6] sequence req6: scanning lock table for conflicting locks +[6] sequence req6: sequencing complete, returned guard + +on-lock-acquired req=req6 key=a dur=r str=intent +---- +[-] acquire lock: txn 00000006 @ ‹a› + +finish req=req6 +---- +[-] finish req6: finishing request + +new-request name=req7 txn=txn6 ts=10,1 + get key=a str=shared +---- + +sequence req=req7 +---- +[7] sequence req7: sequencing request +[7] sequence req7: acquiring latches +[7] sequence req7: scanning lock table for conflicting locks +[7] sequence req7: sequencing complete, returned guard + +on-lock-acquired req=req7 key=a dur=u str=shared +---- +[-] acquire lock: txn 00000006 @ ‹a› + +finish req=req7 +---- +[-] finish req7: finishing request + +debug-lock-table +---- +num=1 + lock: "a" + holder: txn: 00000006-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request name=req8 txn=txn7 ts=10,1 + get key=a +---- + +sequence req=req8 +---- +[8] sequence req8: sequencing request +[8] sequence req8: acquiring latches +[8] sequence req8: scanning lock table for conflicting locks +[8] sequence req8: sequencing complete, returned guard + +handle-lock-conflict-error req=req8 lease-seq=1 + lock txn=txn6 key=a +---- +[9] handle lock conflict error req8: handled conflicting locks on ‹"a"›, released latches + +debug-lock-table +---- +num=1 + lock: "a" + holder: txn: 00000006-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [Intent], unrepl [(str: Shared seq: 0)] + +sequence req=req8 +---- +[10] sequence req8: re-sequencing request +[10] sequence req8: acquiring latches +[10] sequence req8: scanning lock table for conflicting locks +[10] sequence req8: waiting in lock wait-queues +[10] sequence req8: lock wait-queue event: wait for (distinguished) txn 00000006 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) +[10] sequence req8: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false +[10] sequence req8: pushing timestamp of txn 00000006 above 11.000000000,1 +[10] sequence req8: pusher pushed pushee to 11.000000000,2 +[10] sequence req8: resolving intent ‹"a"› for txn 00000006 with PENDING status and clock observation {1 123.000000000,5} +[10] sequence req8: lock wait-queue event: done waiting +[10] sequence req8: conflicted with ‹00000006-0000-0000-0000-000000000000› on ‹"a"› for 0.000s +[10] sequence req8: acquiring latches +[10] sequence req8: scanning lock table for conflicting locks +[10] sequence req8: sequencing complete, returned guard + +finish req=req8 +---- +[-] finish req8: finishing request + +# ------------------------------------------------------------------------------ +# Similar test to the one above, except this time there's both a non-locking +# read and shared locking request waiting on the intent. When the non-locking +# read pushes the intent out of its way, the shared locking request should also +# be allowed to proceed (as we stop tracking the intent in the lock table). +# However, the shared locking request should re-discover the intent. +# ------------------------------------------------------------------------------ + +new-request name=req9 txn=txn8 ts=10,1 + get key=a str=shared +---- + +sequence req=req9 +---- +[11] sequence req9: sequencing request +[11] sequence req9: acquiring latches +[11] sequence req9: scanning lock table for conflicting locks +[11] sequence req9: sequencing complete, returned guard + +handle-lock-conflict-error req=req9 lease-seq=1 + lock txn=txn6 key=a +---- +[12] handle lock conflict error req9: handled conflicting locks on ‹"a"›, released latches + +debug-lock-table +---- +num=1 + lock: "a" + holder: txn: 00000006-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [Intent], unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 9, strength: Shared, txn: 00000008-0000-0000-0000-000000000000 + +sequence req=req9 +---- +[13] sequence req9: re-sequencing request +[13] sequence req9: acquiring latches +[13] sequence req9: scanning lock table for conflicting locks +[13] sequence req9: waiting in lock wait-queues +[13] sequence req9: lock wait-queue event: wait for (distinguished) txn 00000006 holding lock @ key ‹"a"› (queuedLockingRequests: 1, queuedReaders: 0) +[13] sequence req9: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false, wait policy error = false +[13] sequence req9: pushing txn 00000006 to abort +[13] sequence req9: blocked on select in concurrency_test.(*cluster).PushTransaction + +debug-lock-table +---- +num=1 + lock: "a" + holder: txn: 00000006-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [Intent], unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 9, strength: Shared, txn: 00000008-0000-0000-0000-000000000000 + distinguished req: 9 + +# Non-locking request from read committed transaction. It'll be able to +# successfully push the intent because of its isolation level. +new-request name=req10 txn=txn7 ts=10,1 + get key=a +---- + +sequence req=req10 +---- +[13] sequence req9: lock wait-queue event: done waiting +[13] sequence req9: conflicted with ‹00000006-0000-0000-0000-000000000000› on ‹"a"› for 0.000s +[13] sequence req9: acquiring latches +[13] sequence req9: scanning lock table for conflicting locks +[13] sequence req9: sequencing complete, returned guard +[14] sequence req10: sequencing request +[14] sequence req10: acquiring latches +[14] sequence req10: scanning lock table for conflicting locks +[14] sequence req10: waiting in lock wait-queues +[14] sequence req10: lock wait-queue event: wait for txn 00000006 holding lock @ key ‹"a"› (queuedLockingRequests: 1, queuedReaders: 1) +[14] sequence req10: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false +[14] sequence req10: pushing timestamp of txn 00000006 above 11.000000000,1 +[14] sequence req10: resolving intent ‹"a"› for txn 00000006 with PENDING status and clock observation {1 123.000000000,8} +[14] sequence req10: lock wait-queue event: done waiting +[14] sequence req10: conflicted with ‹00000006-0000-0000-0000-000000000000› on ‹"a"› for 0.000s +[14] sequence req10: acquiring latches +[14] sequence req10: scanning lock table for conflicting locks +[14] sequence req10: sequencing complete, returned guard + +debug-lock-table +---- +num=1 + lock: "a" + holder: txn: 00000006-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 9, strength: Shared, txn: 00000008-0000-0000-0000-000000000000 + +finish req=req10 +---- +[-] finish req10: finishing request diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks b/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks index e97a37cd9083..afba1221f135 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks @@ -1141,6 +1141,111 @@ num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] +# ------------------------------------------------------------------------------ +# Tests where both a shared lock and (replicated) intent exist on a key, and the +# intent is forgotten because of a lock update. In such cases, compatible +# waiting requests should be allowed to proceed. They might re-discover the +# intent, but that's okay -- we don't want requests waiting in the lock table if +# the lock that was blocking them isn't being tracked. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req59 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req59 +---- +start-waiting: false + +acquire r=req59 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req60 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req60 +---- +start-waiting: false + +add-discovered k=a durability=r strength=intent txn=txn1 r=req60 +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl [Intent], unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +scan r=req60 +---- +start-waiting: true + +new-request r=req61 txn=txn3 ts=10 spans=shared@a +---- + +scan r=req61 +---- +start-waiting: true + +new-request r=req62 txn=none ts=10 spans=shared@a +---- + +scan r=req62 +---- +start-waiting: true + +new-request r=req63 txn=txn4 ts=10 spans=exclusive@a +---- + +scan r=req63 +---- +start-waiting: true + +new-request r=req64 txn=txn5 ts=10 spans=none@a +---- + +scan r=req64 +---- +start-waiting: true + +new-request r=req65 txn=txn5 ts=10 spans=none@a +---- + +scan r=req65 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl [Intent], unrepl [(str: Shared seq: 0)] + waiting readers: + req: 65, txn: 00000000-0000-0000-0000-000000000005 + req: 64, txn: 00000000-0000-0000-0000-000000000005 + queued locking requests: + active: true req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 61, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 62, strength: Shared, txn: none + active: true req: 63, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 + distinguished req: 60 + +update txn=txn1 ts=11,0 epoch=0 span=a +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 61, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 63, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 + distinguished req: 63 + # TODO(arul): (non-exhaustive list) of shared lock state transitions that aren't # currently supported (and we need to add support for): # diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/update b/pkg/kv/kvserver/concurrency/testdata/lock_table/update index c7ef221e8903..97c51088616b 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/update +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/update @@ -386,6 +386,12 @@ new-lock-table maxlocks=10000 new-txn txn=txn1 ts=10 epoch=1 seq=1 ---- +new-txn txn=txn2 ts=12,1 epoch=0 +---- + +new-txn txn=txn3 ts=14,1 epoch=0 +---- + new-request r=req1 txn=txn1 ts=10 spans=exclusive@a ---- @@ -489,3 +495,69 @@ num=1 update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=1 ---- num=0 + +# ------------------------------------------------------------------------------ +# Tests where both an exclusive lock and a (replicated) intent exists on a key, +# and the intent is forgotten because of a lock update, we correctly let some +# waiting requests through. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req5 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req5 +---- +start-waiting: false + +acquire r=req5 k=a durability=u strength=exclusive +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 10)] + +new-request r=req6 txn=txn2 ts=10 spans=none@a +---- + +scan r=req6 +---- +start-waiting: true + +add-discovered k=a durability=r strength=intent txn=txn1 r=req6 +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 10.000000000,0, info: repl [Intent], unrepl [(str: Exclusive seq: 10)] + waiting readers: + req: 2, txn: 00000000-0000-0000-0000-000000000002 + distinguished req: 2 + +new-request r=req7 txn=txn3 ts=10 spans=shared@a +---- + +scan r=req7 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 10.000000000,0, info: repl [Intent], unrepl [(str: Exclusive seq: 10)] + waiting readers: + req: 2, txn: 00000000-0000-0000-0000-000000000002 + queued locking requests: + active: true req: 3, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + distinguished req: 2 + +update txn=txn1 ts=11,0 epoch=0 span=a +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 10)] + queued locking requests: + active: true req: 3, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + distinguished req: 3