From 5ae79f13b1d942482c1964d4b1b7af6da46832ce Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Sat, 24 Feb 2024 13:57:06 -0500 Subject: [PATCH] concurrency: implement generalized lock promotion Previously, if a request had acquired a shared lock, it wasn't able to promote it to an Exclusive or Intent (by writing to the same key) lock. This was because the lock table could not detect deadlock scenarios where two transactions that both held shared locks were trying to promote. Moreover, it also couldn't detect wait queue local deadlocks that involved non-transactional requests. These two limitations have now been limited. For the former, we're able to leverage our existing deadlock detection algorithm by performing the correct set of pushes. This is done by changing the claimantTxn concept slightly. Previously, there could only be one claimant for a key. This is no longer true -- now, the claimant may be different, depending on who is asking for it. For the latter, we reorder the wait queue to avoid preventable deadlocks. This is done by preferring lock promoters over other requests. The bulk of this was already done in https://github.com/cockroachdb/cockroach/pull/118484. Closes #110435 Release note (sql change): shared locks (acquired using SELECT FOR SHARE or implicitly by read committed transactions) can now be re-acquired with higher strength (using SELECT FOR UPDATE or by writing to the key). --- pkg/kv/kvserver/concurrency/lock_table.go | 271 +++-- .../testdata/lock_table/lock_promotion | 1039 +++++++++++++++++ .../testdata/lock_table/shared_locks | 474 +++----- .../testdata/lock_table/skip_locked | 52 +- 4 files changed, 1407 insertions(+), 429 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 01d4a3c452d3..218babdeebe9 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -56,15 +56,39 @@ const ( waitElsewhere // waitSelf indicates that a different request from the same transaction has - // claimed the lock already. This request should sit tight and wait for a new - // notification without pushing anyone. + // claimed (but not acquired) the lock already. This request should sit tight + // and wait for a new notification without pushing anyone. // - // By definition, the lock cannot be held at this point -- if it were, another - // request from the same transaction would not be in the lock's wait-queues, - // obviating the need for this state. + // It must be the case that there are no lock holders for the associated key + // if a request is actively waiting in the waitSelf state. That's because if + // the key is locked by another transaction, the request must be in the + // waitFor state. A request does not need to actively wait if the only lock + // that's held on the key belongs to its own transaction, thus making the + // waitSelf state impossible for held locks. // - // TODO(arul): this waitSelf state + claimantTxn stuff won't extend well to - // multiple lock holders. See TODO in informActiveWaiters. + // NB: The main motivation of the waitSelf state is to avoid certain + // dependency cycles by potentially re-ordering the lock's wait queue. + // Typically, the waitSelf state arises when there are >= 1 locking requests + // from other transactions between two requests from the same transaction + // waiting in a lock's wait queue. So something like: + // + // key a: unlocked + // wait-queue: [(r1,txn1,inactive), (r2,txn2,active), ... (r3,txn1,active)] + // + // There is a dependency cycle between the requests from txn1 and txn2 here. + // Instead of forcing one of the two transaction's to abort, we're able to + // break the dependency cycle by having r3 wait and sit tight in the waitSelf + // state[1]. Now, if r1 acquires the lock, r3 will be re-ordered[2] to proceed + // before r2 (thus breaking the dependency cycle); if it doesn't, r2 will no + // longer depend on txn1 and can thus proceed (i.e. no more dependency cycle). + // + // [1] A transaction can't push itself, but if it finds the request at the + // head of the queue everyone is pushing belongs to its own transaction, it + // knows there is a local dependency cycle. + // + // [2] Note that re-ordering happens only when r1 acquires the lock, so the + // waitSelf state is a mechanism by which r3 is able to sit tight without + // pushing anyone. waitSelf // waitQueueMaxLengthExceeded indicates that the request attempted to enter a @@ -2009,16 +2033,16 @@ func (kl *keyLocks) informActiveWaiters() { // However, if we naively plugged things into the current structure, it would // either sit tight (because its waiting for itself) or, worse yet, push a // transaction it's actually compatible with! - waitForState.txn, waitForState.held = kl.claimantTxn() for e := kl.waitingReaders.Front(); e != nil; e = e.Next() { state := waitForState + g := e.Value + state.txn, state.held = kl.claimantTxnFor(g) // Since there are waiting readers, we could not have transitioned out of // or into a state where the lock is held. This is because readers only wait // for held locks -- they race with other non-transactional writers and // locking requests. assert(state.held, "waiting readers should be empty if the lock isn't held") - g := e.Value g.mu.Lock() // NB: The waiter is actively waiting on this lock, so it's likely taking // some action based on the previous state (e.g. it may be pushing someone). @@ -2034,9 +2058,12 @@ func (kl *keyLocks) informActiveWaiters() { } g := qg.guard state := waitForState - if g.isSameTxn(waitForState.txn) { - if waitForState.held { - panic("request from the lock holder txn should not be waiting in a wait queue") + state.txn, state.held = kl.claimantTxnFor(g) + if g.isSameTxn(state.txn) { + // The waitSelf state is only possible if the lock isn't held. See the + // comment at the enum declaration for an explanation of why. + if state.held { + panic("requests should not be in the waitSelf state if locks are held") } state.kind = waitSelf } @@ -2050,41 +2077,69 @@ func (kl *keyLocks) informActiveWaiters() { } } -// claimantTxn returns the transaction that the lock table deems as having -// claimed the key. Every lock stored in the lock table must have one and only -// one transaction associated with it that claims the key. All actively waiting -// requests in this lock's wait queues should use this transaction as the -// transaction to push for {liveness,deadlock,etc.} detection purposes related -// to this key. +// claimantTxnFor returns the transaction that the lock table deems to have +// claimed the key, which in turn causes the supplied request to actively wait. +// Transactions that have acquired locks on keys are deemed to have claimed +// them. If a key is not locked, the lock table will deem a request being +// sequenced through the lock table as its next preferred[1] lock holder, and +// coronate it the claimant. +// +// There can be multiple locks on a particular key. There may also be multiple +// compatible (in the common case, compatible with each other) requests being +// sequenced through the lock table. Actively waiting requests are oblivious to +// such details; instead, they use the concept of the claimant transaction as +// the transaction to push for liveness and deadlock detection. In some sense, +// we're using the concept of claimant transactions to collapse the local +// dependency graph represented by the lock's wait queue, but abstracting the +// specifics from the lock table waiter (which is responsible for pushing). // -// The transaction that is considered to have claimed the key may be the lock -// holder, in which case the return value of held will be true. It may also be -// another request being sequenced through the lock table that other waiters -// conflict with. In such cases, this request is deemed to be the next preferred -// lock holder by the lock table. This preference is based on the lock table's -// notion of fairness. +// There is only a single claimant transaction for a key from a request's +// perspective. As such, an actively waiting request only needs to push a single +// transaction at any given time. However, actively waiting requests from +// different transactions may observe different claimants. This is only possible +// if there is an unresolvable[2], local deadlock in the lock's wait queues. +// We're currently not smart enough to recognize this case without engaging +// distributed deadlock detection. Otherwise, all actively waiting requests +// will observe the same claimant on a particular key. // -// Locks on a particular key can be acquired by multiple transactions. There may -// also be multiple (compatible) requests being sequenced through the lock table -// concurrently that can acquire locks on a particular key, one of which will -// win the race. Waiting requests should be oblivious to such details; instead, -// they use the concept of the transaction that has claimed a particular key as -// the transaction to push. +// [1] This preference is based on the lock table's notion of fairness, with +// some re-ordering provisions to avoid deadlocks. See queueOrder for more +// details. +// +// [2] A deadlock is unresolvable if we can't locally re-order the wait queue to +// prevent it. Typically, this happens when 2 shared locks are held on a key, +// and both the shared lock holding transactions are trying to promote their +// locks to either Exclusive or Intent lock strength. // // REQUIRES: kl.mu to be locked. -func (kl *keyLocks) claimantTxn() (_ *enginepb.TxnMeta, held bool) { +func (kl *keyLocks) claimantTxnFor(g *lockTableGuardImpl) (_ *enginepb.TxnMeta, held bool) { if kl.isLocked() { - // We want the claimant transaction to remain the same unless there has been - // a state transition (e.g. the claimant released the lock) that - // necessitates it to change. So we always return the first lock holder, - // ensuring all requests consider the same transaction to have claimed a - // key. - return kl.holders.Front().Value.txn, true + // We want the claimant transaction to be the same for a particular request + // every time this function is called, unless there has been a state + // transition (e.g. the claimant released the lock) that necessitates the + // claimant to change. In cases where there are multiple lock holders on a + // key, we ensure this by always iterating from the start of the lock + // holders list. + for e := kl.holders.Front(); e != nil; e = e.Next() { + // claimantTxnFor is only called by actively waiting requests. Requests + // don't conflict with locks held by their transaction. So, if the lock is + // already held by the supplied request's transaction, then there must be + // another conflicting lock held by a different transaction that's causing + // it to actively wait on this key -- in such cases, this other + // transaction is the claimant from the supplied request's perspective. + // This prompts the check below. + if !g.isSameTxn(e.Value.txn) { + return e.Value.txn, true + } + } } if kl.queuedLockingRequests.Len() == 0 { panic("no queued locking request or lock holder; no one should be waiting on the lock") } qg := kl.queuedLockingRequests.Front().Value + // NB: the returned transaction can be the same transaction to which the + // supplied request belongs, and that's okay. See the waitSelf state for more + // details. return qg.guard.txnMeta(), false } @@ -2374,7 +2429,7 @@ func (kl *keyLocks) constructWaitingState(g *lockTableGuardImpl) waitingState { queuedReaders: kl.waitingReaders.Len(), held: true, } - txn, held := kl.claimantTxn() + txn, held := kl.claimantTxnFor(g) waitForState.held = held waitForState.txn = txn if g.isSameTxn(waitForState.txn) { @@ -2389,6 +2444,7 @@ func (kl *keyLocks) constructWaitingState(g *lockTableGuardImpl) waitingState { // compared to the one supplied. Otherwise, false is returned. // // REQUIRES: kl.mu to be locked. +// TODO(arul): We can get rid of the error path here. func (kl *keyLocks) alreadyHoldsLockAndIsAllowedToProceed( g *lockTableGuardImpl, str lock.Strength, ) (bool, error) { @@ -2404,10 +2460,6 @@ func (kl *keyLocks) alreadyHoldsLockAndIsAllowedToProceed( } tl := e.Value heldMode := tl.getLockMode() - err := kl.maybeDisallowLockPromotion(heldMode.Strength, str) - if err != nil { - return false, err - } // Check if the lock is already held by the guard's transaction with an equal // or higher lock strength. If it is, we're good to go. Otherwise, the request // is trying to promote a lock it previously acquired. In such cases, the @@ -2417,26 +2469,6 @@ func (kl *keyLocks) alreadyHoldsLockAndIsAllowedToProceed( return str <= heldMode.Strength, nil } -// maybeDisallowLockPromotion checks if a lock is being promoted from -// lock.Shared to lock.Intent/lock.Exclusive, and returns an error if that's the -// case. See: https://github.com/cockroachdb/cockroach/issues/110435. -// -// REQUIRES: kl.mu to be locked. -// -// TODO(arul): Once we handle lock promotion correctly, and this function goes -// away, a lot of the error paths where this error is bubbled up to can go away. -func (kl *keyLocks) maybeDisallowLockPromotion( - held lock.Strength, reAcquisitionStr lock.Strength, -) error { - if held == lock.Shared && reAcquisitionStr > held { - return MarkLockPromotionError(errors.UnimplementedErrorf( - errors.IssueLink{IssueURL: "https://github.com/cockroachdb/cockroach/issues/110435"}, - "lock promotion from %s to %s is not allowed", held, reAcquisitionStr, - )) - } - return nil -} - // conflictsWithLockHolders returns true if the request, referenced by the // supplied lockTableGuardImpl, conflicts with any of the locks held on this // key. Non-conflicting requests are allowed to proceed; conflicting requests @@ -2662,15 +2694,6 @@ func (kl *keyLocks) insertLockingRequest( if qqg.order.after(qg.order) { break } - if qg.guard.txn != nil && qqg.guard.isSameTxn(qg.guard.txnMeta()) { - // There's another request, from our transaction, that's waiting to acquire - // a lock on this key before us. As per its sequence number, it'll get a - // chance to do so before we do -- assuming it'll succeed, check if we've - // got ourselves into a lock promotion case that's not allowed. - if err := kl.maybeDisallowLockPromotion(qqg.mode.Strength, qg.mode.Strength); err != nil { - return nil, err - } - } } if e == nil { kl.queuedLockingRequests.PushBack(qg) @@ -2888,6 +2911,17 @@ func (kl *keyLocks) acquireLock( // break claims of requests that hold latches without holding latches // themselves. + // NB: The lock may have been acquired with a lower strength than the strength + // with which other requests from the (now) lock holder transaction are trying + // to access this key. Put another way, some requests may now lock promoters + // as a result of this lock acquisition. Instead of marking them as inactive, + // we simply release them from the lock's wait queue and have them re-scan. + // This ensures they enqueue in the correct place (promoters get preference) + // and correctly determine whether they need to wait or not at this key. + // + // TODO(arul): Replace the call to releaseLockingRequestsFromTxn with + // recomputeWaitQueue, and push the responsibility of re-ordering the wait + // queue into there. Ditto for the call in addDiscoveredLock. kl.releaseLockingRequestsFromTxn(&acq.Txn) // Sanity check that there aren't any waiting readers on this lock. There @@ -3124,16 +3158,49 @@ func (kl *keyLocks) tryUpdateLockLocked( // update. gc = kl.releaseWaitersOnKeyUnlocked() } else { - // If we're in this branch, it must be the case that there are multiple - // shared locks held on this key, and as a result, releasing one of the - // locks hasn't transitioned the key to unlocked. However, the lock that - // we just released may have belonged to the claimant transaction -- the - // one that any waiters on this key were pushing. If this is the case, - // we'll need to inform these waiters about a new claimant they should be - // pushing instead. A call to informActiveWaiters will do exactly that. - // Note that if the lock that was cleared didn't belong to a transaction - // all waiters were pushing, the call to informActiveWaiters will no-op. - kl.informActiveWaiters() + // In some rare cases (such as replays), there may be requests belonging + // to the transaction that just released its lock in the lock's wait + // queue. These must be requests that are promoting their lock to + // something stronger, as otherwise they wouldn't be in the wait queue + // to begin with. However, now that the lock has been released, the + // queueOrder.isPromoting determination changes as well. We could handle + // this two different ways: + // 1. Recompute queueOrder.isPromoting for waiting requests that belong to + // the transaction that just released the lock and reorder the wait queue + // (if needed). + // 2. Release any locking requests that belong to the transaction that + // just released its lock. They'll re-scan, re-determine whether they're + // promoting or not, and get inserted in the correct spot. We choose the + // latter option. + // + // TODO(arul): The former option, where we push this complexity into + // recomputeWaitQueues, is better. We should switch to that, and in doing + // so, get rid of all calls to releaseLockingRequestsFromTxn and replace + // them with recomputeWaitQueues. Notably, this includes the call to that + // method in the lock acquisition path as well. + kl.releaseLockingRequestsFromTxn(&up.Txn) + // If we're in this branch, it must be the case that there were multiple + // shared locks held on this key. + // + // [1] Releasing one of the locks hasn't transitioned the key to unlocked. + // However, the lock we just released might have been the one some of the + // waiters were pushing, in which case they need to push a new guy. + // + // [2] Moreover, any requests that belong to a transaction that holds a + // shared lock that were waiting to promote their lock to something + // stronger might be able to do so now. + // + // As such, we need to recompute the wait queue for the lock. However, [2] + // is only possible if there's only one shared lock left on this key. + // Otherwise, the promoting request (if present) will conflict with the + // other shared lock. Moreover, for [1], simply calling + // informActiveWaiters will do. As recomputeWaitQueues is O(number of lock + // holder), we can optimize this to the if condition below: + if kl.holders.Len() == 1 { + kl.recomputeWaitQueues(st) + } else { + kl.informActiveWaiters() + } } return true, gc } @@ -3218,7 +3285,16 @@ func (kl *keyLocks) tryUpdateLockLocked( if !kl.isLocked() { gc = kl.releaseWaitersOnKeyUnlocked() } else { - kl.informActiveWaiters() + // See the handling of finalized transactions that release a shared lock + // above for an explanation. Releasing a lock due to ignored sequence + // numbers is no different than releasing a lock because the holder's + // transaction has been finalized. + kl.releaseLockingRequestsFromTxn(&up.Txn) + if kl.holders.Len() == 1 { + kl.recomputeWaitQueues(st) + } else { + kl.informActiveWaiters() + } } return true, gc } @@ -3236,10 +3312,13 @@ func (kl *keyLocks) tryUpdateLockLocked( // recomputeWaitQueues goes through the receiver's wait queues and recomputes // whether actively waiting requests should continue to do so, given the key's -// locks holders and other waiting requests. Such computation is necessary when -// a lock's strength has decreased[1] or locking requests have dropped out of -// wait queue's[2] without actually acquiring the lock or the lock's timestamp -// has advanced. +// locks holders and other waiting requests. A non-exhaustive list scenarios where +// such computation is necessary is: +// - A lock's strength has decreased[1] +// - Or locking requests have dropped out of wait queue's[2] without actually +// acquiring the lock +// - Or the lock's timestamp has advanced +// - Or one of the locks on a key has been released[3] // // [1] This can happen as a result of savepoint rollback or when the lock table // stops tracking a replicated lock because of a PUSH_TIMESTAMP that @@ -3249,6 +3328,11 @@ func (kl *keyLocks) tryUpdateLockLocked( // already in the lock's wait queue. Locking requests dropping out of a lock's // wait queue can therefore result in other requests no longer needing to // actively wait. +// [3] If there are multiple shared locks on a key and on of the transactions +// that holds the shared lock is trying to promote its strength to something +// stronger, then the release of a shared lock may allow it to do so. Note that +// this can only happen if there's one shared lock (the one belonging to the +// promoting transaction) left. // // TODO(arul): We could optimize this function if we had information about the // context it was being called in. @@ -3260,12 +3344,18 @@ func (kl *keyLocks) recomputeWaitQueues(st *cluster.Settings) { // only considered None, Shared, Exclusive, and Intent locking strengths // (which are the only ones that exist at the time of writing). var strongestMode lock.Mode + var lockHolderTxn *enginepb.TxnMeta // nil if {no,more than one} locks are held // Go through the list of lock holders. // TODO(arul): We should annotate each of the holders to reflect if the lock // belongs to a finalized transaction or not. If it does, we should exclude // it when computing strongestLockMode. for e := kl.holders.Front(); e != nil; e = e.Next() { mode := e.Value.getLockMode() + if strongestMode.Empty() { + lockHolderTxn = e.Value.txn + } else { + lockHolderTxn = nil + } if strongestMode.Weaker(mode) { strongestMode = mode } @@ -3293,7 +3383,10 @@ func (kl *keyLocks) recomputeWaitQueues(st *cluster.Settings) { qlr := e.Value curr := e e = e.Next() - if lock.Conflicts(qlr.mode, strongestMode, &st.SV) { + belongsToOnlyLockHolder := lockHolderTxn != nil && qlr.guard.isSameTxn(lockHolderTxn) + conflictsWithEarlier := lock.Conflicts(qlr.mode, strongestMode, &st.SV) && + !belongsToOnlyLockHolder + if conflictsWithEarlier { break } removed := false @@ -3832,7 +3925,7 @@ func (kl *keyLocks) verify(st *cluster.Settings) error { // 6. Verify the waiting state on each of the waiters. for e := kl.waitingReaders.Front(); e != nil; e = e.Next() { - claimantTxn, _ := kl.claimantTxn() + claimantTxn, _ := kl.claimantTxnFor(e.Value) e.Value.mu.Lock() if e.Value.mu.state.kind == waitSelf { return errors.AssertionFailedf("readers should never wait for themselves") @@ -3848,7 +3941,7 @@ func (kl *keyLocks) verify(st *cluster.Settings) error { // actively waiting requests. continue } - claimantTxn, _ := kl.claimantTxn() + claimantTxn, _ := kl.claimantTxnFor(e.Value.guard) e.Value.guard.mu.Lock() if e.Value.guard.isSameTxn(claimantTxn) && e.Value.guard.mu.state.kind != waitSelf { return errors.AssertionFailedf("locking request should be in waitSelf") diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion index 7233ad452192..b450f48bdaee 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion @@ -145,3 +145,1042 @@ num=1 active: false req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + +# ------------------------------------------------------------------------------ +# Basic test for lock promotion from Shared -> Exclusive, when there are no +# other waiters on the lock and no other shared locks held on the key. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req8 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req8 +---- +start-waiting: false + +acquire r=req8 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=req9 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req9 +---- +start-waiting: false + +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: 9, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +acquire r=req9 k=a durability=u strength=exclusive +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: false req: 9, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +dequeue r=req9 +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + +# ------------------------------------------------------------------------------ +# Test for lock promotion from Shared -> Exclusive, when there are other waiters +# on the lock but no other shared locks held on the key. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req10 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req10 +---- +start-waiting: false + +acquire r=req10 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=req11 txn=txn2 ts=10 spans=exclusive@a +---- + +scan r=req11 +---- +start-waiting: true + + +new-request r=req12 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req12 +---- +start-waiting: false + +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: 12, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +acquire r=req12 k=a durability=u strength=exclusive +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: false req: 12, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +dequeue r=req12 +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: true req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +# ------------------------------------------------------------------------------ +# Test for lock promotion from Shared -> Exclusive when there are 2 shared locks +# held on the key. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req13 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req13 +---- +start-waiting: false + +acquire r=req13 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=req14 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req14 +---- +start-waiting: false + +acquire r=req14 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req15 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req15 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 15, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +guard-state r=req15 +---- +new: state=waitFor txn=txn2 key="a" held=true guard-strength=Exclusive + +release txn=txn2 span=a +---- +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: 15, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +# ------------------------------------------------------------------------------ +# Test for lock promotion from Shared -> Exclusive when there are 3 shared locks +# held on the key. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req16 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req16 +---- +start-waiting: false + +acquire r=req16 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=req17 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req17 +---- +start-waiting: false + +acquire r=req17 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req18 txn=txn3 ts=10 spans=shared@a +---- + +scan r=req18 +---- +start-waiting: false + +acquire r=req18 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req19 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req19 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +guard-state r=req19 +---- +new: state=waitFor txn=txn2 key="a" held=true guard-strength=Exclusive + +release txn=txn2 span=a +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +guard-state r=req19 +---- +new: state=waitFor txn=txn3 key="a" held=true guard-strength=Exclusive + +release txn=txn3 span=a +---- +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: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +guard-state r=req19 +---- +new: state=doneWaiting + +# Re-scan to simulate how the lock table waiter will handle this scenario. +scan r=req19 +---- +start-waiting: false + +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: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +acquire r=req19 k=a durability=u strength=exclusive +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: false req: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +dequeue r=req19 +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + +# ------------------------------------------------------------------------------ +# Test for lock promotion from Shared -> Exclusive which causes a deadlock. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req20 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req20 +---- +start-waiting: false + +acquire r=req20 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=req21 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req21 +---- +start-waiting: false + +acquire r=req21 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req22 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req22 +---- +start-waiting: true + +guard-state r=req22 +---- +new: state=waitFor txn=txn2 key="a" held=true guard-strength=Exclusive + +new-request r=req23 txn=txn2 ts=10 spans=exclusive@a +---- + +scan r=req23 +---- +start-waiting: true + +guard-state r=req23 +---- +new: state=waitFor txn=txn1 key="a" held=true guard-strength=Exclusive + +# We should now be to detect a deadlock -- note the guard states above. + +# ------------------------------------------------------------------------------ +# Test the case where a shared lock holder is trying to promote and there's a +# non-transactional conflicting request already waiting at the lock. The +# promoting request should be able to jump ahead; not doing so would constitute +# a deadlock that we wouldn't be able to detect -- that's because our +# distributed deadlock detection algorithm assumes non-transactional requests +# can never be part of deadlock cycles. +# +# NB: This construction is what prompted the concept of queueOrder. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req24 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req24 +---- +start-waiting: false + +acquire r=req24 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=req25 txn=none ts=10 spans=intent@a +---- + +scan r=req25 +---- +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: 25, strength: Intent, txn: none + +new-request r=req26 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req26 +---- +start-waiting: false + +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: 26, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 25, strength: Intent, txn: none + +acquire r=req26 k=a durability=u strength=exclusive +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: false req: 26, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 25, strength: Intent, txn: none + +dequeue r=req26 +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: true req: 25, strength: Intent, txn: none + + +# ------------------------------------------------------------------------------ +# Test a scenario where a deadlock is possible given the state of the wait +# queues, but because no locks have been acquired yet, it won't be detected. +# Instead, one of the requests will wait in the waitSelf states. +# +# Key A: unlocked +# Wait Queue: [shared txn1, shared txn2, exclusive txn1, exclusive txn2] +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req27 txn=txn3 ts=10 spans=exclusive@a +---- + +scan r=req27 +---- +start-waiting: false + +acquire r=req27 k=a durability=u strength=exclusive +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + +new-request r=req28 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req28 +---- +start-waiting: true + +new-request r=req29 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req29 +---- +start-waiting: true + +release txn=txn3 span=a +---- +num=1 + lock: "a" + queued locking requests: + active: false req: 28, strength: Shared, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 29, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +new-request r=req30 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req30 +---- +start-waiting: true + +guard-state r=req30 +---- +new: state=waitSelf + +new-request r=req31 txn=txn2 ts=10 spans=exclusive@a +---- + +scan r=req31 +---- +start-waiting: true + +guard-state r=req31 +---- +new: state=waitFor txn=txn1 key="a" held=false guard-strength=Exclusive + +print +---- +num=1 + lock: "a" + queued locking requests: + active: false req: 28, strength: Shared, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 29, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 30, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 31, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +# There's a possible deadlock, but we're not going to detect it just yet, +# because no locks have been actually acquired. Let's change that. + +acquire r=req28 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)] + queued locking requests: + active: false req: 29, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 31, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +# Note that req29 needs to acquire the lock (or start waiting elsewhere) before +# req30 (which has now been released) can re-scan. That's because req29 still +# holds latches. We test the "acquire lock" scenario here. + +acquire r=req29 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +# req30 and req31 are now uncorked and should re-scan. +guard-state r=req30 +---- +new: state=doneWaiting + +scan r=req30 +---- +start-waiting: true + +guard-state r=req31 +---- +new: state=doneWaiting + +scan r=req31 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 30, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 31, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +guard-state r=req30 +---- +new: state=waitFor txn=txn2 key="a" held=true guard-strength=Exclusive + +guard-state r=req31 +---- +new: state=waitFor txn=txn1 key="a" held=true guard-strength=Exclusive + +# Deadlock detected. + +# ------------------------------------------------------------------------------ +# Very similar test to above, except this time, instead of testing the "acquire +# lock" scenario, we'll instead test the "wait elsewhere and drop latches" case. +# ------------------------------------------------------------------------------ + + +clear +---- +num=0 + +new-request r=req32 txn=txn3 ts=10 spans=exclusive@a+exclusive@b +---- + +scan r=req32 +---- +start-waiting: false + +acquire r=req32 k=a durability=u strength=exclusive +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + +acquire r=req32 k=b durability=u strength=exclusive +---- +num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + +new-request r=req33 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req33 +---- +start-waiting: true + +new-request r=req34 txn=txn2 ts=10 spans=shared@a+shared@b +---- + +scan r=req34 +---- +start-waiting: true + +release txn=txn3 span=a +---- +num=2 + lock: "a" + queued locking requests: + active: false req: 33, strength: Shared, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + +new-request r=req35 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req35 +---- +start-waiting: true + +guard-state r=req35 +---- +new: state=waitSelf + +new-request r=req36 txn=txn2 ts=10 spans=exclusive@a +---- + +scan r=req36 +---- +start-waiting: true + +guard-state r=req36 +---- +new: state=waitFor txn=txn1 key="a" held=false guard-strength=Exclusive + +print +---- +num=2 + lock: "a" + queued locking requests: + active: false req: 33, strength: Shared, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 35, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + +# There's a possible deadlock, but we're not going to detect it just yet, +# because no locks have been actually acquired. Let's change that. + +guard-state r=req33 +---- +new: state=doneWaiting + +acquire r=req33 k=a durability=u strength=shared +---- +num=2 + 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: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + + +# Note that req34 needs to acquire the lock (or start waiting elsewhere) before +# req30 (which has now been released) can re-scan. That's because req34 still +# holds latches. We tested the "acquire lock" case above; we'll now test the +# "wait elsewhere" (on key b) case. + +guard-state r=req34 +---- +new: state=waitFor txn=txn3 key="b" held=true guard-strength=Shared + +scan r=req34 +---- +start-waiting: true + +print +---- +num=2 + 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: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + queued locking requests: + active: true req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +# req35 is uncorked. +guard-state r=req35 +---- +new: state=doneWaiting + +scan r=req35 +---- +start-waiting: false + +guard-state r=req36 +---- +new: state=waitFor txn=txn1 key="a" held=true guard-strength=Exclusive + +print +---- +num=2 + 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: 35, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + queued locking requests: + active: true req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +acquire r=req35 k=a durability=u strength=exclusive +---- +num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: false req: 35, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + queued locking requests: + active: true req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +dequeue r=req35 +---- +num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] + queued locking requests: + active: true req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +# Uncork req34 by releasing the lock on b. It previously held a joint claim on +# key a, but that claim was broken by req35. It'll notice this when it re-scans. + +release txn=txn3 span=b +---- +num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + queued locking requests: + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +guard-state r=req34 +---- +new: state=doneWaiting + +scan r=req34 +---- +start-waiting: true + +print +---- +num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0), (str: Shared seq: 0)] + queued locking requests: + active: true req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 36, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + lock: "b" + queued locking requests: + active: false req: 34, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + +# Deadlock prevented because req35 broke req34's joint claim on key a. + +# ------------------------------------------------------------------------------ +# Test for lock promotion from Shared -> Exclusive when there are 2 shared locks +# on the key and one of the locks is released as a result of sequence number +# rollbacks. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req37 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req37 +---- +start-waiting: false + +acquire r=req37 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=req38 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req38 +---- +start-waiting: false + +acquire r=req38 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req39 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req39 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 39, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +guard-state r=req39 +---- +new: state=waitFor txn=txn2 key="a" held=true guard-strength=Exclusive + +update txn=txn2 ts=10 epoch=1 span=a ignored-seqs=1 +---- +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: 39, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +# ------------------------------------------------------------------------------ +# Test that when a request transitions from promoting to non-promoting (because +# a lock is released because of ignored sequence numbers), the wait queue is +# re-ordered correctly. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req40 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req40 +---- +start-waiting: false + +acquire r=req40 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=req41 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req41 +---- +start-waiting: false + +acquire r=req41 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req42 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req42 +---- +start-waiting: true + +new-request r=req43 txn=txn2 ts=10 spans=exclusive@a +---- + +scan r=req43 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 42, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 43, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=0 +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 43, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +scan r=req42 +---- +start-waiting: true + +scan r=req43 +---- +start-waiting: false + +print +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 43, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 42, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + +# ------------------------------------------------------------------------------ +# Same test as above, but this time we release the lock by committing the +# transaction instead of rolling back sequence numbers. +# ------------------------------------------------------------------------------ + +clear +---- +num=0 + +new-request r=req44 txn=txn1 ts=10 spans=shared@a +---- + +scan r=req44 +---- +start-waiting: false + +acquire r=req44 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req45 txn=txn2 ts=10 spans=shared@a +---- + +scan r=req45 +---- +start-waiting: false + +acquire r=req45 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req46 txn=txn1 ts=10 spans=exclusive@a +---- + +scan r=req46 +---- +start-waiting: true + +new-request r=req47 txn=txn2 ts=10 spans=exclusive@a +---- + +scan r=req47 +---- +start-waiting: true + +print +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: true req: 46, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 47, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +release txn=txn1 span=a +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 47, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + +scan r=req46 +---- +start-waiting: true + +scan r=req47 +---- +start-waiting: false + +print +---- +num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + queued locking requests: + active: false req: 47, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 46, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks b/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks index 19d80d8e2bdb..8bf3e5edd053 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks @@ -907,208 +907,6 @@ num=1 active: true req: 45, strength: Shared, txn: 00000000-0000-0000-0000-000000000001 active: true req: 46, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 -# ------------------------------------------------------------------------------ -# Test for lock promotion. When a lock is held with strength Shared, we do not -# allow the holder to promote it to Exclusive or write to the key. The same -# applies if the lock isn't held, but there's a request from our transaction -# trying to acquire a Shared lock (that's waiting in front of us). -# ------------------------------------------------------------------------------ - -clear ----- -num=0 - -new-request r=req47 txn=txn1 ts=10 spans=shared@b ----- - -scan r=req47 ----- -start-waiting: false - -acquire k=b r=req47 strength=shared durability=u ----- -num=1 - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - -new-request r=req48 txn=txn1 ts=10 spans=exclusive@b ----- - -scan r=req48 ----- -lock promotion from Shared to Exclusive is not allowed - -new-request r=req49 txn=txn1 ts=10 spans=intent@b ----- - -scan r=req49 ----- -lock promotion from Shared to Intent is not allowed - -new-request r=req50 txn=txn2 ts=10 spans=exclusive@b ----- - -scan r=req50 ----- -start-waiting: true - -new-request r=req51 txn=txn1 ts=10 spans=exclusive@a ----- - -scan r=req51 ----- -start-waiting: false - -acquire r=req51 k=a durability=u strength=exclusive ----- -num=2 - lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - queued locking requests: - active: true req: 50, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 - -new-request r=req52 txn=txn3 ts=10 spans=exclusive@a+exclusive@b ----- - -scan r=req52 ----- -start-waiting: true - -new-request r=req53 txn=txn3 ts=10 spans=shared@b ----- - -scan r=req53 ----- -start-waiting: true - -print ----- -num=2 - lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued locking requests: - active: true req: 52, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - queued locking requests: - active: true req: 50, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 53, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - -new-request r=req54 txn=txn3 ts=10 spans=exclusive@b ----- - -scan r=req54 ----- -lock promotion from Shared to Exclusive is not allowed - -new-request r=req55 txn=txn3 ts=10 spans=intent@b ----- - -scan r=req55 ----- -lock promotion from Shared to Intent is not allowed - -release txn=txn1 span=a ----- -num=2 - lock: "a" - queued locking requests: - active: false req: 52, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - queued locking requests: - active: true req: 50, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 53, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - -scan r=req52 ----- -start-waiting: true - -print ----- -num=2 - lock: "a" - queued locking requests: - active: false req: 52, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - queued locking requests: - active: true req: 50, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 52, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 53, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - -# ------------------------------------------------------------------------------ -# Sub-test for lock promotion. When an initial call to ScanAndEnqueue returns an -# error, the request is dequeued from any other wait-queues that it may have -# entered. -# ------------------------------------------------------------------------------ - -clear ----- -num=0 - -new-request r=req56 txn=txn1 ts=10 spans=exclusive@a ----- - -scan r=req56 ----- -start-waiting: false - -acquire r=req56 k=a durability=u strength=exclusive ----- -num=1 - lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - -dequeue r=req56 ----- -num=1 - lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - -new-request r=req57 txn=txn2 ts=10 spans=shared@b ----- - -scan r=req57 ----- -start-waiting: false - -acquire r=req57 k=b durability=u strength=shared ----- -num=2 - lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - -dequeue r=req57 ----- -num=2 - lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - -# Mark txn1 as aborted so that the next two requests immediately acquire claims -# on key "a" when scanning. -pushed-txn-updated txn=txn1 status=aborted ----- - -new-request r=req58 txn=txn2 ts=10 spans=exclusive@a,c ----- - -scan r=req58 ----- -lock promotion from Shared to Exclusive is not allowed - -print ----- -num=1 - lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] - # ------------------------------------------------------------------------------ # Tests where both a shared lock and (replicated) intent exist on a key, and the # intent is forgotten because of a lock update. In such cases, compatible @@ -1121,70 +919,70 @@ clear ---- num=0 -new-request r=req59 txn=txn1 ts=10 spans=shared@a +new-request r=req47 txn=txn1 ts=10 spans=shared@a ---- -scan r=req59 +scan r=req47 ---- start-waiting: false -acquire r=req59 k=a durability=u strength=shared +acquire r=req47 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=req60 txn=txn2 ts=10 spans=shared@a +new-request r=req48 txn=txn2 ts=10 spans=shared@a ---- -scan r=req60 +scan r=req48 ---- start-waiting: false -add-discovered k=a durability=r strength=intent txn=txn1 r=req60 +add-discovered k=a durability=r strength=intent txn=txn1 r=req48 ---- num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl [Intent], unrepl [(str: Shared seq: 0)] queued locking requests: - active: false req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 48, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 -scan r=req60 +scan r=req48 ---- start-waiting: true -new-request r=req61 txn=txn3 ts=10 spans=shared@a +new-request r=req49 txn=txn3 ts=10 spans=shared@a ---- -scan r=req61 +scan r=req49 ---- start-waiting: true -new-request r=req62 txn=none ts=10 spans=shared@a +new-request r=req50 txn=none ts=10 spans=shared@a ---- -scan r=req62 +scan r=req50 ---- start-waiting: true -new-request r=req63 txn=txn4 ts=10 spans=exclusive@a +new-request r=req51 txn=txn4 ts=10 spans=exclusive@a ---- -scan r=req63 +scan r=req51 ---- start-waiting: true -new-request r=req64 txn=txn5 ts=10 spans=none@a +new-request r=req52 txn=txn5 ts=10 spans=none@a ---- -scan r=req64 +scan r=req52 ---- start-waiting: true -new-request r=req65 txn=txn5 ts=10 spans=none@a +new-request r=req53 txn=txn5 ts=10 spans=none@a ---- -scan r=req65 +scan r=req53 ---- start-waiting: true @@ -1194,13 +992,13 @@ num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl [Intent], unrepl [(str: Shared seq: 0)] waiting readers: - req: 65, txn: 00000000-0000-0000-0000-000000000005 - req: 64, txn: 00000000-0000-0000-0000-000000000005 + req: 53, txn: 00000000-0000-0000-0000-000000000005 + req: 52, txn: 00000000-0000-0000-0000-000000000005 queued locking requests: - active: true req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 61, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 62, strength: Shared, txn: none - active: true req: 63, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 + active: true req: 48, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 49, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 50, strength: Shared, txn: none + active: true req: 51, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 update txn=txn1 ts=11,0 epoch=0 span=a ---- @@ -1208,9 +1006,9 @@ 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: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 - active: false req: 61, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 63, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 + active: false req: 48, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 49, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 51, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 # ------------------------------------------------------------------------------ # Test when a locking request drops out of a wait queue and makes other actively @@ -1223,37 +1021,37 @@ clear ---- num=0 -new-request r=req66 txn=txn1 ts=10 spans=shared@a +new-request r=req54 txn=txn1 ts=10 spans=shared@a ---- -scan r=req66 +scan r=req54 ---- start-waiting: false -acquire r=req66 k=a durability=u strength=shared +acquire r=req54 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 +new-request r=req55 txn=txn2 ts=10 spans=exclusive@a ---- -scan r=req67 +scan r=req55 ---- start-waiting: true -new-request r=req68 txn=txn3 ts=10 spans=shared@a +new-request r=req56 txn=txn3 ts=10 spans=shared@a ---- -scan r=req68 +scan r=req56 ---- start-waiting: true -new-request r=req69 txn=txn4 ts=10 spans=shared@a +new-request r=req57 txn=txn4 ts=10 spans=shared@a ---- -scan r=req69 +scan r=req57 ---- start-waiting: true @@ -1263,18 +1061,18 @@ 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 + active: true req: 55, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 56, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 57, strength: Shared, txn: 00000000-0000-0000-0000-000000000004 -dequeue r=req67 +dequeue r=req55 ---- 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 + active: false req: 56, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: false req: 57, strength: Shared, txn: 00000000-0000-0000-0000-000000000004 print ---- @@ -1282,8 +1080,8 @@ 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 + active: false req: 56, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: false req: 57, strength: Shared, txn: 00000000-0000-0000-0000-000000000004 # ------------------------------------------------------------------------------ # Regression test for @@ -1294,50 +1092,50 @@ clear ---- num=0 -new-request r=req70 txn=txn1 ts=10 spans=intent@a +new-request r=req58 txn=txn1 ts=10 spans=intent@a ---- -scan r=req70 +scan r=req58 ---- start-waiting: false -new-request r=req71 txn=none ts=10 spans=intent@a +new-request r=req59 txn=none ts=10 spans=intent@a ---- -scan r=req71 +scan r=req59 ---- start-waiting: false -add-discovered r=req71 txn=txn2 k=a durability=u strength=shared +add-discovered r=req59 txn=txn2 k=a durability=u strength=shared ---- num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: repl [Shared] queued locking requests: - active: false req: 71, strength: Intent, txn: none + active: false req: 59, strength: Intent, txn: none -scan r=req70 +scan r=req58 ---- start-waiting: true -new-request r=req72 txn=txn3 ts=10 spans=shared@a +new-request r=req60 txn=txn3 ts=10 spans=shared@a ---- -scan r=req72 +scan r=req60 ---- start-waiting: true -new-request r=req73 txn=none ts=10 spans=shared@a +new-request r=req61 txn=none ts=10 spans=shared@a ---- -scan r=req73 +scan r=req61 ---- start-waiting: true -new-request r=req74 txn=none ts=10 spans=intent@a +new-request r=req62 txn=none ts=10 spans=intent@a ---- -scan r=req74 +scan r=req62 ---- start-waiting: true @@ -1347,30 +1145,30 @@ num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: repl [Shared] queued locking requests: - active: true req: 70, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 - active: false req: 71, strength: Intent, txn: none - active: true req: 72, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 73, strength: Shared, txn: none - active: true req: 74, strength: Intent, txn: none + active: true req: 58, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 59, strength: Intent, txn: none + active: true req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 61, strength: Shared, txn: none + active: true req: 62, strength: Intent, txn: none release txn=txn2 span=a ---- num=1 lock: "a" queued locking requests: - active: false req: 70, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 - active: false req: 71, strength: Intent, txn: none - active: true req: 72, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 73, strength: Shared, txn: none - active: true req: 74, strength: Intent, txn: none + active: false req: 58, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 + active: false req: 59, strength: Intent, txn: none + active: true req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 61, strength: Shared, txn: none + active: true req: 62, strength: Intent, txn: none -dequeue r=req70 +dequeue r=req58 ---- num=1 lock: "a" queued locking requests: - active: false req: 72, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 74, strength: Intent, txn: none + active: false req: 60, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 62, strength: Intent, txn: none # ------------------------------------------------------------------------------ # Ensure when a lock is acquired, and we release other locking requests from the @@ -1381,27 +1179,27 @@ clear ---- num=0 -new-request r=req75 txn=txn3 ts=10 spans=exclusive@a +new-request r=req63 txn=txn3 ts=10 spans=exclusive@a ---- -scan r=req75 +scan r=req63 ---- start-waiting: false -acquire r=req75 k=a durability=u strength=exclusive +acquire r=req63 k=a durability=u strength=exclusive ---- num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] -new-request r=req76 txn=txn1 ts=10 spans=shared@b +new-request r=req64 txn=txn1 ts=10 spans=shared@b ---- -scan r=req76 +scan r=req64 ---- start-waiting: false -acquire r=req76 k=b durability=u strength=shared +acquire r=req64 k=b durability=u strength=shared ---- num=2 lock: "a" @@ -1409,17 +1207,17 @@ num=2 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] -new-request r=req77 txn=txn2 ts=10 spans=shared@a+shared@b +new-request r=req65 txn=txn2 ts=10 spans=shared@a+shared@b ---- -scan r=req77 +scan r=req65 ---- start-waiting: true -new-request r=req78 txn=txn2 ts=10 spans=exclusive@b +new-request r=req66 txn=txn2 ts=10 spans=exclusive@b ---- -scan r=req78 +scan r=req66 ---- start-waiting: true @@ -1429,24 +1227,24 @@ num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] queued locking requests: - active: true req: 77, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 65, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: true req: 78, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 66, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 release txn=txn3 span=a ---- num=2 lock: "a" queued locking requests: - active: false req: 77, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 65, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: true req: 78, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 66, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 -scan r=req77 +scan r=req65 ---- start-waiting: false @@ -1455,17 +1253,17 @@ print num=2 lock: "a" queued locking requests: - active: false req: 77, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 65, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: false req: 77, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 78, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 65, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 66, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 -new-request r=req79 txn=txn4 ts=10 spans=shared@b +new-request r=req67 txn=txn4 ts=10 spans=shared@b ---- -scan r=req79 +scan r=req67 ---- start-waiting: true @@ -1474,34 +1272,34 @@ print num=2 lock: "a" queued locking requests: - active: false req: 77, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 65, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: false req: 77, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 78, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 79, strength: Shared, txn: 00000000-0000-0000-0000-000000000004 + active: false req: 65, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 66, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 67, strength: Shared, txn: 00000000-0000-0000-0000-000000000004 # The shared lock acquisition by request 77 should remove request 78 from the # wait queue as they both belong to txn 2. Request 78 will detect it's trying to # promote its lock and get an error (shown after). However, once request 78 is # removed, there's no reason for request 79 to actively wait at this key. It # should acquire a claim by marking itself as inactive. -acquire r=req77 k=b durability=u strength=shared +acquire r=req65 k=b durability=u strength=shared ---- num=2 lock: "a" queued locking requests: - active: false req: 77, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 65, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: false req: 79, strength: Shared, txn: 00000000-0000-0000-0000-000000000004 + active: false req: 67, strength: Shared, txn: 00000000-0000-0000-0000-000000000004 -scan r=req78 +scan r=req66 ---- -lock promotion from Shared to Exclusive is not allowed +start-waiting: true # ----------------------------------------------------------------------------- # Test when a non-transactional request finds a lock held by a finalized @@ -1516,80 +1314,80 @@ clear ---- num=0 -new-request r=req80 txn=txn3 ts=10 spans=shared@b +new-request r=req68 txn=txn3 ts=10 spans=shared@b ---- -scan r=req80 +scan r=req68 ---- start-waiting: false -acquire r=req80 k=b durability=u strength=shared +acquire r=req68 k=b durability=u strength=shared ---- num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] -new-request r=req81 txn=txn1 ts=10 spans=shared@a +new-request r=req69 txn=txn1 ts=10 spans=shared@a ---- -scan r=req81 +scan r=req69 ---- start-waiting: false -acquire r=req81 k=a durability=r strength=shared +acquire r=req69 k=a durability=r strength=shared ---- num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] -new-request r=req82 txn=none ts=10 spans=exclusive@a+exclusive@b +new-request r=req70 txn=none ts=10 spans=exclusive@a+exclusive@b ---- -scan r=req82 +scan r=req70 ---- start-waiting: true -add-discovered r=req82 k=a txn=txn1 strength=shared +add-discovered r=req70 k=a txn=txn1 strength=shared ---- num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: repl [Shared] queued locking requests: - active: false req: 82, strength: Exclusive, txn: none + active: false req: 70, strength: Exclusive, txn: none lock: "b" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: true req: 82, strength: Exclusive, txn: none + active: true req: 70, strength: Exclusive, txn: none -guard-state r=req82 +guard-state r=req70 ---- new: state=waitFor txn=txn3 key="b" held=true guard-strength=Exclusive -new-request r=req83 txn=none ts=10 spans=shared@a +new-request r=req71 txn=none ts=10 spans=shared@a ---- -scan r=req83 +scan r=req71 ---- start-waiting: true -new-request r=req84 txn=txn2 ts=10 spans=shared@a +new-request r=req72 txn=txn2 ts=10 spans=shared@a ---- -scan r=req84 +scan r=req72 ---- start-waiting: true -new-request r=req85 txn=txn3 ts=10 spans=shared@a +new-request r=req73 txn=txn3 ts=10 spans=shared@a ---- -scan r=req85 +scan r=req73 ---- start-waiting: true -new-request r=req86 txn=txn4 ts=10 spans=exclusive@a +new-request r=req74 txn=txn4 ts=10 spans=exclusive@a ---- -scan r=req86 +scan r=req74 ---- start-waiting: true @@ -1599,15 +1397,15 @@ num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: repl [Shared] queued locking requests: - active: false req: 82, strength: Exclusive, txn: none - active: true req: 83, strength: Shared, txn: none - active: true req: 84, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 85, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 86, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 + active: false req: 70, strength: Exclusive, txn: none + active: true req: 71, strength: Shared, txn: none + active: true req: 72, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 73, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 74, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: true req: 82, strength: Exclusive, txn: none + active: true req: 70, strength: Exclusive, txn: none pushed-txn-updated txn=txn1 status=aborted ---- @@ -1618,17 +1416,17 @@ num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: repl [Shared] [holder finalized: aborted] queued locking requests: - active: false req: 82, strength: Exclusive, txn: none - active: true req: 83, strength: Shared, txn: none - active: true req: 84, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 - active: true req: 85, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 86, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 + active: false req: 70, strength: Exclusive, txn: none + active: true req: 71, strength: Shared, txn: none + active: true req: 72, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: true req: 73, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 74, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: true req: 82, strength: Exclusive, txn: none + active: true req: 70, strength: Exclusive, txn: none -scan r=req82 +scan r=req70 ---- start-waiting: true @@ -1638,10 +1436,10 @@ num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: repl [Shared] [holder finalized: aborted] queued locking requests: - active: false req: 84, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 - active: false req: 85, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 - active: true req: 86, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 + active: false req: 72, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 + active: false req: 73, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 + active: true req: 74, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] queued locking requests: - active: true req: 82, strength: Exclusive, txn: none + active: true req: 70, strength: Exclusive, txn: none diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked index 7c323b7c7f9f..f702df802187 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked @@ -355,11 +355,11 @@ locked: true, holder: 00000000-0000-0000-0000-000000000001 is-key-locked-by-conflicting-txn r=req8 k=h strength=exclusive ---- -lock promotion from Shared to Exclusive is not allowed +locked: false is-key-locked-by-conflicting-txn r=req8 k=i strength=exclusive ---- -lock promotion from Shared to Exclusive is not allowed +locked: false dequeue r=req8 ---- @@ -474,3 +474,51 @@ false is-key-locked-by-conflicting-txn r=req10 k=f strength=exclusive ---- SKIP LOCKED request should not find another waiting request from the same transaction + +# ------------------------------------------------------------------------------ +# Ensure SKIP LOCKED works correctly when a transaction holds one of multiple +# shared locks on a key, and it tries to use SKIP LOCKED in on the key with a +# stronger lock strength (i.e. it's trying to promote its lock). +# --------------------------------------------------------------------------------- + +clear +---- +num=0 + +new-request r=req11 txn=txn1 ts=10,1 spans=shared@a +---- + +scan r=req11 +---- +start-waiting: false + +acquire r=req11 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=req12 txn=txn2 ts=10,1 spans=shared@a +---- + +scan r=req12 +---- +start-waiting: false + +acquire r=req12 k=a durability=u strength=shared +---- +num=1 + lock: "a" + holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)] + +new-request r=req13 txn=txn2 ts=10,1 spans=exclusive@a skip-locked +---- + +scan r=req13 +---- +start-waiting: false + +is-key-locked-by-conflicting-txn r=req13 k=a strength=exclusive +---- +locked: true, holder: 00000000-0000-0000-0000-000000000001