Skip to content

Commit

Permalink
concurrency: always list for lock state transitions when pushing
Browse files Browse the repository at this point in the history
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:

- 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 the 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
  • Loading branch information
arulajmani committed Oct 20, 2023
1 parent 72c01ad commit eb33878
Show file tree
Hide file tree
Showing 3 changed files with 120 additions and 56 deletions.
54 changes: 35 additions & 19 deletions pkg/kv/kvserver/concurrency/lock_table_waiter.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -379,6 +360,41 @@ func (w *lockTableWaiterImpl) WaitOn(
pushCtx, pushCancel := context.WithCancel(ctx)
defer pushCancel()
go watchForNotifications(pushCtx, pushCancel, newStateC)

// Note that even though the request conflicts with a held lock, the
// lock's state may still be updated[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 lock may be rolled back because of savepoints even if the
// transaction isn't finalized/pushed successfully.
// 2. 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.
if timerWaitingState.held {
err = w.pushLockTxn(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) {
err = nil
}
return err
}

// 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)
if errors.Is(pushCtx.Err(), context.Canceled) {
// Ignore the context canceled error. If this was for the
Expand Down
Loading

0 comments on commit eb33878

Please sign in to comment.