Skip to content

Commit

Permalink
concurrency: recompute wait queues when locking requests drop out
Browse files Browse the repository at this point in the history
A locking request must actively wait in a lock's wait queues if:
- it conflicts with any of the lock holders.
- or it conflicts with a lower sequence numbered request already in
the lock's wait queue.

As a result, if a locking request exits a lock's wait queue without
actually acquiring the lock, it may allow other locking requests to
proceed. This patch recomputes wait queues whenever a locking request
exits a lock's wait queues to detect such scenarios and unblock requests
which were actively waiting previously not no longer need to.

Fixes #111144

Release note: None
  • Loading branch information
arulajmani committed Nov 1, 2023
1 parent 554d6e0 commit 97213a3
Show file tree
Hide file tree
Showing 2 changed files with 85 additions and 7 deletions.
17 changes: 10 additions & 7 deletions pkg/kv/kvserver/concurrency/lock_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -3495,13 +3495,16 @@ func (kl *keyLocks) requestDone(g *lockTableGuardImpl) (gc bool) {
}
}

if !kl.isLocked() && doneRemoval {
// The first request in the queuedLockingRequests should always be an
// inactive, transactional locking request if the lock isn't held. That may
// no longer be true if the guy we removed above was serving this purpose;
// the call to maybeReleaseCompatibleLockingRequests should fix that. And if
// it wasn't serving that purpose, it'll be a no-op.
kl.maybeReleaseCompatibleLockingRequests()
if doneRemoval {
// If a locking request is removed from the receiver's wait queues without
// actually acquiring a lock other locking requests may be able to proceed.
// This is because locking requests must actively wait if they conflict with
// either the lock holder or any lower sequence numbered locking requests.
// In cases where request(s) were waiting just because they conflicted with
// the request we just removed, and they are compatible with all lock
// holders (if any), we need to let them go. A call to recomputeWaitQueues
// will do exactly that, and will be a no-op otherwise.
kl.recomputeWaitQueues(g.lt.settings)
}

if !doneRemoval {
Expand Down
75 changes: 75 additions & 0 deletions pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks
Original file line number Diff line number Diff line change
Expand Up @@ -1246,6 +1246,81 @@ num=1
active: true req: 63, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004
distinguished req: 63

# ------------------------------------------------------------------------------
# Test when a locking request drops out of a wait queue and makes other actively
# waiting requests compatible as a result. They should be able to proceed.
# Serves as a regression test for
# https://github.com/cockroachdb/cockroach/issues/111144.
# ------------------------------------------------------------------------------

clear
----
num=0

new-request r=req66 txn=txn1 ts=10 spans=shared@a
----

scan r=req66
----
start-waiting: false

acquire r=req66 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=req67 txn=txn2 ts=10 spans=exclusive@a
----

scan r=req67
----
start-waiting: true

new-request r=req68 txn=txn3 ts=10 spans=shared@a
----

scan r=req68
----
start-waiting: true

new-request r=req69 txn=txn4 ts=10 spans=shared@a
----

scan r=req69
----
start-waiting: true

print
----
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: true req: 67, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002
active: true req: 68, strength: Shared, txn: 00000000-0000-0000-0000-000000000003
active: true req: 69, strength: Shared, txn: 00000000-0000-0000-0000-000000000004
distinguished req: 67

dequeue r=req67
----
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: 68, strength: Shared, txn: 00000000-0000-0000-0000-000000000003
active: false req: 69, strength: Shared, txn: 00000000-0000-0000-0000-000000000004

print
----
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: 68, strength: Shared, txn: 00000000-0000-0000-0000-000000000003
active: false req: 69, strength: Shared, txn: 00000000-0000-0000-0000-000000000004


# TODO(arul): (non-exhaustive list) of shared lock state transitions that aren't
# currently supported (and we need to add support for):
#
Expand Down

0 comments on commit 97213a3

Please sign in to comment.