Skip to content

Commit

Permalink
concurrency: pull sequence number comparisons behind methods
Browse files Browse the repository at this point in the history
The lock table uses sequence numbers (which are a function of request
arrival order) to determine the sort order for requests in a lock's
wait queue. We'll expand this criteria soon to enable lock promotion.
In preparation, we pull raw sequence number comparisons behind methods
on a newly introduced `queueOrder` struct.

Epic: none

Release note: None
  • Loading branch information
arulajmani committed Feb 7, 2024
1 parent 826145d commit df82b0e
Showing 1 changed file with 50 additions and 19 deletions.
69 changes: 50 additions & 19 deletions pkg/kv/kvserver/concurrency/lock_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -741,10 +741,10 @@ func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn(

for e := l.queuedLockingRequests.Front(); e != nil; e = e.Next() {
qqg := e.Value
if qqg.guard.seqNum > g.seqNum {
// We only need to check for conflicts with requests that came before us
// (read: have lower sequence numbers than us). Note that the list of
// queuedLockingRequests is sorted in increasing order of sequence number.
qo := makeQueueOrder(g)
if qqg.order.after(qo) {
// We only need to check for conflicts with requests that sort before us.
// Note that the list of queuedLockingRequests is already sorted.
break
}
if qqg.guard.txnMeta() != nil && g.isSameTxn(qqg.guard.txnMeta()) {
Expand Down Expand Up @@ -1000,6 +1000,33 @@ type queuedGuard struct {
guard *lockTableGuardImpl
mode lock.Mode // protected by keyLocks.mu
active bool // protected by keyLocks.mu
order queueOrder
}

// queueOrder encapsulates fields that are used to determine the order in which
// locking requests are stored in a lock's wait queue.
type queueOrder struct {
reqSeqNum uint64
}

// makeQueueOrder constructs a queueOrder.
func makeQueueOrder(g *lockTableGuardImpl) queueOrder {
return queueOrder{
reqSeqNum: g.seqNum,
}
}

// after returns true if the receiver should be ordered after the supplied
// queueOrder.
//
// Comparison is based on sequence numbers, which correspond to a request's
// arrival time -- requests that arrive later are ordered after requests that
// arrive earlier, and vice-versa.
func (o1 queueOrder) after(o2 queueOrder) bool {
if o1.reqSeqNum == o2.reqSeqNum {
return false // same request; doesn't sort after
}
return o1.reqSeqNum > o2.reqSeqNum
}

// Information about a lock holder for unreplicated locks.
Expand Down Expand Up @@ -2722,8 +2749,8 @@ func (kl *keyLocks) enqueueLockingRequest(
}

// insertLockingRequest inserts the locking request, trying to access the lock
// with the supplied strength, at the correct position into the lock's wait
// queue. The request is wrapped in a queuedGuard to insert it into the queue,
// with the supplied strength, at the correct position in the lock's wait queue.
// The request is wrapped in a queuedGuard to insert it into the queue,
// the reference for which is returned to the caller.
//
// Note that the request should not already be present in the lock's wait queue.
Expand All @@ -2744,13 +2771,18 @@ func (kl *keyLocks) insertLockingRequest(
guard: g,
mode: makeLockMode(accessStrength, g.txnMeta(), g.ts),
active: true,
order: makeQueueOrder(g),
}
// The request isn't in the queue. Add it in the correct position, based on
// its sequence number.
// its queueOrder.
var e *list.Element[*queuedGuard]
// TODO(arul): Once we've addressed lifted the limitation on lock promotion
// (https://github.com/cockroachdb/cockroach/issues/110435), we should switch
// this loop to iterate from the back of the queue -- it's more likely that
// new requests are added to the back of the queue.
for e = kl.queuedLockingRequests.Front(); e != nil; e = e.Next() {
qqg := e.Value
if qqg.guard.seqNum > qg.guard.seqNum {
if qqg.order.after(qg.order) {
break
}
if qg.guard.txn != nil && qqg.guard.isSameTxn(qg.guard.txnMeta()) {
Expand Down Expand Up @@ -3859,12 +3891,12 @@ func (kl *keyLocks) verify(st *cluster.Settings) error {
}
}

// 2. Ensure queued locking requests are stored in sorted sequence number
// order.
for e1 := kl.queuedLockingRequests.Front(); e1 != nil; e1 = e1.Next() {
if e1.Prev() != nil && e1.Prev().Value.guard.seqNum >= e1.Value.guard.seqNum {
// 2. Ensure queued locking requests are stored in the correct order, as
// dictated by the queuedGuard.order field.
for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() {
if e.Prev() != nil && e.Prev().Value.order.after(e.Value.order) {
return errors.AssertionFailedf(
"queued locking requests should be stored in sequence number order %s", kl,
"queued locking requests should be stored in sorted order %s", kl,
)
}
}
Expand Down Expand Up @@ -3918,8 +3950,7 @@ func (kl *keyLocks) verify(st *cluster.Settings) error {
}
// If a locking request is actively waiting at a key, it must either:
// 1. Conflict with one of the lock holders.
// 2. OR Conflict with one of the queued locking requests in front (read:
// lower sequence number) of it.
// 2. OR Conflict with one of the queued locking requests in front of it.
conflicts := false
for e2 := kl.holders.Front(); e2 != nil; e2 = e2.Next() {
holder := e2.Value
Expand All @@ -3936,11 +3967,11 @@ func (kl *keyLocks) verify(st *cluster.Settings) error {
// No conflict found with lock holder(s); check other queued locking
// requests waiting in front of the request.
for e2 := kl.queuedLockingRequests.Front(); ; e2 = e2.Next() {
req := e2.Value
if req.guard.seqNum >= qlr.guard.seqNum {
break
qg := e2.Value
if qg.guard == qlr.guard {
break // we've found our request
}
if lock.Conflicts(req.mode, qlr.mode, &st.SV) {
if lock.Conflicts(qg.mode, qlr.mode, &st.SV) {
conflicts = true
break
}
Expand Down

0 comments on commit df82b0e

Please sign in to comment.