From 51a3ce4a131eb504601e550801fbc405952cb773 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Fri, 20 Oct 2023 12:31:43 -0500 Subject: [PATCH] concurrency: always listen for lock state transitions when pushing Prior to this patch, the lockTableWaiter would only listen for lock state transitions if it was pushing a transaction while waiting for an unheld lock. The reasoning was that if the lock was held, the pusher would not be able to proceed until the push returned. This isn't quite true -- there's a few cases where the request may no longer conflict with what's being tracked in the lock table: - A waiting request may not conflict with the lock holder, but a waiting request instead. In such cases, it'll push the lock holder even though it is compatible with it. However, if the conflicting waiting request drops out of the lock's wait queues, the pusher may proceed. - The lock may have been rolled back because of savepoints. - The lock may have been forgotten by the lock table (replicated locks are forgotten when they're updated). This patch changes the lockTableWaiter to also listen for state transitions when pushing a held lock's transaction. Cases where the pusher no longer conflicts with the lock state are detected and the push is cancelled. Conveniently, the updates to `resolve_pushed_intents` show the effect of making this change. Fixes #111596 Release note: None --- .../kvserver/concurrency/lock_table_waiter.go | 85 ++++++++----- .../resolve_pushed_intents | 112 +++++++++++++----- .../concurrency_manager/wait_elsewhere | 14 +-- 3 files changed, 144 insertions(+), 67 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index bba7c71feb5d..75f70a422f05 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -137,7 +137,7 @@ type IntentResolver interface { // WaitOn implements the lockTableWaiter interface. func (w *lockTableWaiterImpl) WaitOn( ctx context.Context, req Request, guard lockTableGuard, -) (err *Error) { +) *Error { newStateC := guard.NewStateChan() ctxDoneC := ctx.Done() shouldQuiesceC := w.stopper.ShouldQuiesce() @@ -351,25 +351,6 @@ func (w *lockTableWaiterImpl) WaitOn( // push with the option to wait on the conflict if active. pushWait := func(ctx context.Context) *Error { - // If the request is conflicting with a held lock then it pushes its - // holder synchronously - there is no way it will be able to proceed - // until the lock's transaction undergoes a state transition (either - // completing or being pushed) and then updates the lock's state - // through intent resolution. The request has a dependency on the - // entire conflicting transaction. - // - // However, if the request is conflicting with another request (that has - // claimed the lock, but not yet acquired it) then it pushes the - // claimant transaction asynchronously while continuing to listen to - // state transition in the lockTable. This allows the request to cancel - // its push if the conflicting claimant transaction exits the lock - // wait-queue without leaving behind a lock. In this case, the request - // has a dependency on the conflicting request but not necessarily the - // entire conflicting transaction. - if timerWaitingState.held { - return w.pushLockTxn(ctx, req, timerWaitingState) - } - // It would be more natural to launch an async task for the push and // continue listening on this goroutine for lockTable state transitions, // but doing so is harder to test against. Instead, we launch an async @@ -379,15 +360,62 @@ func (w *lockTableWaiterImpl) WaitOn( pushCtx, pushCancel := context.WithCancel(ctx) defer pushCancel() go watchForNotifications(pushCtx, pushCancel, newStateC) - err := w.pushRequestTxn(pushCtx, req, timerWaitingState) - if errors.Is(pushCtx.Err(), context.Canceled) { - // Ignore the context canceled error. If this was for the - // parent context then we'll notice on the next select. + + var err *Error + if timerWaitingState.held { + // Note that even though the request has a dependency on the + // transaction that holds the lock, this dependency can be broken + // without the holder's transaction getting finalized[1] such that the + // pusher can proceed before the synchronous push below returns. The + // pusher must detect such cases (watchForNotifications) and cancel + // its push in such cases. + // + // [1] This can happen for a few reasons: + // 1. The pusher may not conflict with the lock holder itself, but one + // of the waiting requests instead. If the waiting request drops out + // of the lock's wait queue the pusher should be allowed to proceed. + // Concretely, a construction like follows: + // - holder: shared + // - wait-queue: exclusive, shared + // In this case, the waiting shared lock request will push the + // holder[*] However, if the waiting exclusive locking request drops + // out of the wait queue, the shared locking request no longer needs + // to wait/push the holder. + // 2. The lock may be rolled back because of savepoints even if the + // transaction isn't finalized/pushed successfully. + // 3. The lock may no longer be tracked by the lock table even though + // the holder's transaction is still pending. This can happen if it's + // an intent that's pushed to a higher timestamp by a different + // request. In such cases, the lock table will simply forget the lock + // when the intent is resolved. Note that in such cases, the pusher + // may still conflict with the intent and rediscover it -- that's + // okay. // - // NOTE: we look at pushCtx.Err() and not err to avoid the - // potential for bugs if context cancellation is not - // propagated correctly on some error paths. - err = nil + // [*] The shared locking request will push the lock holder (strength + // shared) instead of the exclusive lock requesting (the one it + // actually conflicts with) because it transitively depends on the + // shared locking request. In doing so, it is essentially collapsing + // edges in the local portion of its dependency graph for deadlock + // detection, as doing so is cheaper that finding out the same + // information using (QueryTxnRequest) RPCs. + err = w.pushLockTxn(pushCtx, req, timerWaitingState) + } else { + // The request conflicts with another request that's claimed an unheld + // lock. The conflicting request may exit the lock table without + // actually acquiring the lock. If that happens, we may be able to + // proceed without needing to wait for the push to successfully + // complete. Such cases will be detected by listening for lock state + // transitions (watchForNotifications). + err = w.pushRequestTxn(pushCtx, req, timerWaitingState) + } + // Ignore the context canceled error. If this was for the parent context + // then we'll notice on the next select. + // + // NOTE: we look at pushCtx.Err() and not err to avoid the potential for + // bugs if context cancellation is not propagated correctly on some + // error paths. + if errors.Is(pushCtx.Err(), context.Canceled) { + return nil } return err } @@ -402,6 +430,7 @@ func (w *lockTableWaiterImpl) WaitOn( // We push with or without the option to wait on the conflict, // depending on the state of the lock timeout, if one exists, // and depending on the wait policy. + var err *Error if req.WaitPolicy == lock.WaitPolicy_Error { err = w.pushLockTxn(ctx, req, timerWaitingState) } else if !lockDeadline.IsZero() { diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents index 31a8edb62669..88e3ff95b9bb 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents @@ -62,9 +62,11 @@ num=10 # Before re-scanning and pushing, add a waiter on a single key to demonstrate # that uncontended, replicated keys are released when pushed, while contended, -# replicated keys are not. +# replicated keys are not. We use a shared locking waiter here so that our high +# priority scan doesn't conflict on latches with it (once the replicated lock +# stops being tracked). new-request name=req2 txn=txn3 ts=10,1 - put key=c value=val + get key=c str=shared ---- sequence req=req2 @@ -81,6 +83,11 @@ sequence req=req2 # Now re-scan with the high-priority reader. sequence req=req1 ---- +[3] sequence req2: lock wait-queue event: done waiting +[3] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s +[3] sequence req2: acquiring latches +[3] sequence req2: scanning lock table for conflicting locks +[3] sequence req2: sequencing complete, returned guard [4] sequence req1: re-sequencing request [4] sequence req1: acquiring latches [4] sequence req1: scanning lock table for conflicting locks @@ -110,23 +117,40 @@ finish req=req1 ---- [-] finish req1: finishing request +handle-lock-conflict-error req=req2 lease-seq=1 + lock txn=txn2 key=c +---- +[5] handle lock conflict error req2: handled conflicting locks on ‹"c"›, released latches + # Only the contended lock remains. debug-lock-table ---- num=1 lock: "c" + holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [Intent] queued locking requests: - active: false req: 2, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 + active: false req: 2, strength: Shared, txn: 00000003-0000-0000-0000-000000000000 + +sequence req=req2 +---- +[6] sequence req2: re-sequencing request +[6] sequence req2: acquiring latches +[6] sequence req2: scanning lock table for conflicting locks +[6] sequence req2: waiting in lock wait-queues +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) +[6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false, wait policy error = false +[6] sequence req2: pushing txn 00000002 to abort +[6] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn2 status=aborted ---- [-] update txn: aborting txn2 -[3] sequence req2: resolving intent ‹"c"› for txn 00000002 with ABORTED status -[3] sequence req2: lock wait-queue event: done waiting -[3] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s -[3] sequence req2: acquiring latches -[3] sequence req2: scanning lock table for conflicting locks -[3] sequence req2: sequencing complete, returned guard +[6] sequence req2: resolving intent ‹"c"› for txn 00000002 with ABORTED status +[6] sequence req2: lock wait-queue event: done waiting +[6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s +[6] sequence req2: acquiring latches +[6] sequence req2: scanning lock table for conflicting locks +[6] sequence req2: sequencing complete, returned guard finish req=req2 ---- @@ -218,7 +242,7 @@ sequence req=req1 [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: pusher pushed pushee to 10.000000000,2 -[4] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,4} +[4] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,5} [4] sequence req1: lock wait-queue event: done waiting [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"a"› for 0.000s [4] sequence req1: resolving a batch of 1 intent(s) @@ -292,17 +316,17 @@ sequence req=req1 [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [3] sequence req1: pushing timestamp of txn 00000002 above 11.000000000,1 [3] sequence req1: pusher pushed pushee to 11.000000000,2 -[3] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,6} +[3] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,7} [3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"a"› for 0.000s [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [3] sequence req1: pushing timestamp of txn 00000002 above 11.000000000,1 -[3] sequence req1: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,8} +[3] sequence req1: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,9} [3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [3] sequence req1: pushing timestamp of txn 00000002 above 11.000000000,1 -[3] sequence req1: resolving intent ‹"c"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,10} +[3] sequence req1: resolving intent ‹"c"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,11} [3] sequence req1: lock wait-queue event: done waiting [3] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s [3] sequence req1: acquiring latches @@ -388,9 +412,11 @@ num=10 # Before re-scanning and pushing, add a waiter on a single key to demonstrate # that uncontended, replicated keys are released when pushed, while contended, -# replicated keys are not. +# replicated keys are not. We use a shared locking waiter here so that our high +# priority scan doesn't conflict on latches with it (once the replicated lock +# stops being tracked). new-request name=req2 txn=txn3 ts=10,1 - put key=c value=val + get key=c str=shared ---- sequence req=req2 @@ -407,6 +433,11 @@ sequence req=req2 # Now re-scan with the high-priority reader. sequence req=req1 ---- +[3] sequence req2: lock wait-queue event: done waiting +[3] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s +[3] sequence req2: acquiring latches +[3] sequence req2: scanning lock table for conflicting locks +[3] sequence req2: sequencing complete, returned guard [4] sequence req1: re-sequencing request [4] sequence req1: acquiring latches [4] sequence req1: scanning lock table for conflicting locks @@ -415,52 +446,52 @@ sequence req=req1 [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: pusher pushed pushee to 10.000000000,2 -[4] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,13} +[4] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,14} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"a"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,15} +[4] sequence req1: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,16} [4] sequence req1: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"c"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,17} +[4] sequence req1: resolving intent ‹"c"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,18} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"d"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"d"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,19} +[4] sequence req1: resolving intent ‹"d"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,20} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"e"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"d"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"e"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,21} +[4] sequence req1: resolving intent ‹"e"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,22} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"f"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"e"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"f"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,23} +[4] sequence req1: resolving intent ‹"f"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,24} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"g"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"f"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"g"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,25} +[4] sequence req1: resolving intent ‹"g"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,26} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"h"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"g"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"h"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,27} +[4] sequence req1: resolving intent ‹"h"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,28} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"i"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"h"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"i"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,29} +[4] sequence req1: resolving intent ‹"i"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,30} [4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"j"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"i"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true, wait policy error = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 -[4] sequence req1: resolving intent ‹"j"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,31} +[4] sequence req1: resolving intent ‹"j"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,32} [4] sequence req1: lock wait-queue event: done waiting [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"j"› for 0.000s [4] sequence req1: acquiring latches @@ -471,23 +502,40 @@ finish req=req1 ---- [-] finish req1: finishing request +handle-lock-conflict-error req=req2 lease-seq=1 + lock txn=txn2 key=c +---- +[5] handle lock conflict error req2: handled conflicting locks on ‹"c"›, released latches + # Only the contended lock remains. debug-lock-table ---- num=1 lock: "c" + holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [Intent] queued locking requests: - active: false req: 7, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 + active: false req: 7, strength: Shared, txn: 00000003-0000-0000-0000-000000000000 + +sequence req=req2 +---- +[6] sequence req2: re-sequencing request +[6] sequence req2: acquiring latches +[6] sequence req2: scanning lock table for conflicting locks +[6] sequence req2: waiting in lock wait-queues +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) +[6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false, wait policy error = false +[6] sequence req2: pushing txn 00000002 to abort +[6] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction on-txn-updated txn=txn2 status=aborted ---- [-] update txn: aborting txn2 -[3] sequence req2: resolving intent ‹"c"› for txn 00000002 with ABORTED status -[3] sequence req2: lock wait-queue event: done waiting -[3] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s -[3] sequence req2: acquiring latches -[3] sequence req2: scanning lock table for conflicting locks -[3] sequence req2: sequencing complete, returned guard +[6] sequence req2: resolving intent ‹"c"› for txn 00000002 with ABORTED status +[6] sequence req2: lock wait-queue event: done waiting +[6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s +[6] sequence req2: acquiring latches +[6] sequence req2: scanning lock table for conflicting locks +[6] sequence req2: sequencing complete, returned guard finish req=req2 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere index d6dc92d1b95d..29c77d203be9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere @@ -84,6 +84,9 @@ sequence req=reqSecondLock on-lock-acquired req=reqSecondLock key=k2 dur=u ---- [-] acquire lock: txn 00000001 @ ‹k2› +[4] sequence reqWaiter: lock wait-queue event: wait elsewhere for txn 00000001 @ key ‹"k"› +[4] sequence reqWaiter: pushing txn 00000001 to abort +[4] sequence reqWaiter: blocked on select in concurrency_test.(*cluster).PushTransaction finish req=reqSecondLock ---- @@ -94,15 +97,12 @@ debug-advance-clock ts=123 # Abort the writing txn. This will cause the blocked request to unblock. Note # that we expect the "conflicted with" contention event after the push. This -# shows that the event is emitted only after the request exits both the waitFor -# and the waitElsewhere states. +# shows that the event is emitted only after the request exits the waitFor +# state. on-txn-updated txn=txnWriter status=aborted ---- [-] update txn: aborting txnWriter [4] sequence reqWaiter: resolving intent ‹"k"› for txn 00000001 with ABORTED status -[4] sequence reqWaiter: lock wait-queue event: wait elsewhere for txn 00000001 @ key ‹"k"› -[4] sequence reqWaiter: pushing txn 00000001 to abort -[4] sequence reqWaiter: resolving intent ‹"k"› for txn 00000001 with ABORTED status [4] sequence reqWaiter: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"k"› for 123.000s [4] sequence reqWaiter: acquiring latches [4] sequence reqWaiter: scanning lock table for conflicting locks @@ -256,13 +256,13 @@ num=1 on-txn-updated txn=txnThreeKeyWriter status=pending ts=20,2 ---- [-] update txn: increasing timestamp of txnThreeKeyWriter -[10] sequence reqTwoKeyWaiter: resolving intent ‹"k1"› for txn 00000003 with PENDING status and clock observation {1 246.000000000,2} +[10] sequence reqTwoKeyWaiter: resolving intent ‹"k1"› for txn 00000003 with PENDING status and clock observation {1 246.000000000,1} [10] sequence reqTwoKeyWaiter: lock wait-queue event: done waiting [10] sequence reqTwoKeyWaiter: conflicted with ‹00000003-0000-0000-0000-000000000000› on ‹"k1"› for 0.000s [10] sequence reqTwoKeyWaiter: acquiring latches [10] sequence reqTwoKeyWaiter: scanning lock table for conflicting locks [10] sequence reqTwoKeyWaiter: sequencing complete, returned guard -[12] sequence reqThreeKeyWaiter: resolving intent ‹"k1"› for txn 00000003 with PENDING status and clock observation {1 246.000000000,4} +[12] sequence reqThreeKeyWaiter: resolving intent ‹"k1"› for txn 00000003 with PENDING status and clock observation {1 246.000000000,3} [12] sequence reqThreeKeyWaiter: lock wait-queue event: done waiting [12] sequence reqThreeKeyWaiter: conflicted with ‹00000003-0000-0000-0000-000000000000› on ‹"k1"› for 0.000s [12] sequence reqThreeKeyWaiter: acquiring latches