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