From f7d6e66ea0c93062bc0bd30470c6ffaa3827938d Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Thu, 24 Aug 2023 15:56:15 -0400 Subject: [PATCH 1/2] concurrency: s/queuedWriters/queuedLockingRequests/g The new name is more apt for what is being stored in this list. No functional change, but I did clean up some commentary here. Epic: none Release note: None --- pkg/kv/kvserver/concurrency/lock_table.go | 459 +++++++++--------- .../kvserver/concurrency/lock_table_test.go | 14 +- .../testdata/concurrency_manager/basic | 8 +- .../clear_abandoned_intents | 68 +-- ...doned_intents_without_adding_to_lock_table | 2 +- .../testdata/concurrency_manager/deadlocks | 82 ++-- .../discover_lock_after_lease_race | 4 +- .../concurrency_manager/discovered_lock | 2 +- .../concurrency_manager/isolation_level | 32 +- .../testdata/concurrency_manager/lock_timeout | 20 +- .../testdata/concurrency_manager/optimistic | 2 +- .../testdata/concurrency_manager/priority | 30 +- .../concurrency_manager/queue_length_exceeded | 22 +- .../concurrency_manager/range_state_listener | 46 +- .../resolve_pushed_intents | 38 +- ...ushed_intents_without_adding_to_lock_table | 42 +- .../testdata/concurrency_manager/uncertainty | 10 +- .../testdata/concurrency_manager/update | 10 +- .../concurrency_manager/wait_elsewhere | 8 +- .../concurrency_manager/wait_policy_error | 20 +- .../concurrency_manager/wait_policy_skip | 4 +- .../testdata/concurrency_manager/wait_self | 12 +- .../testdata/lock_table/acquire_ignored_seqs | 8 +- .../testdata/lock_table/add_discovered | 8 +- .../concurrency/testdata/lock_table/basic | 160 +++--- .../concurrency/testdata/lock_table/clear | 4 +- .../lock_table/clear_finalized_txn_locks | 168 +++---- .../concurrency/testdata/lock_table/disable | 6 +- .../discovered_locks_consults_txn_cache | 12 +- .../testdata/lock_table/dup_access | 102 ++-- .../testdata/lock_table/lock_changes | 2 +- .../testdata/lock_table/lock_dropped | 4 +- .../testdata/lock_table/non_active_waiter | 42 +- .../testdata/lock_table/non_txn_write | 52 +- .../concurrency/testdata/lock_table/query | 4 +- .../testdata/lock_table/queue_length_exceeded | 18 +- .../testdata/lock_table/shared_locks | 17 +- .../testdata/lock_table/size_limit_exceeded | 20 +- .../testdata/lock_table/skip_locked | 20 +- .../concurrency/testdata/lock_table/update | 20 +- .../concurrency/testdata/lock_table/wait_self | 8 +- 41 files changed, 806 insertions(+), 804 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 9065e3f1710e..c48ef0f51f8d 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -75,9 +75,9 @@ const ( waitSelf // waitQueueMaxLengthExceeded indicates that the request attempted to enter a - // lock wait-queue as a writer and found that the queue's length was already - // equal to or exceeding the request's configured maximum. As a result, the - // request was rejected. + // lock wait-queue as a locking request and found that the queue's length was + // already equal to or exceeding the request's configured maximum. As a + // result, the request was rejected. waitQueueMaxLengthExceeded // doneWaiting indicates that the request is done waiting on this pass @@ -96,11 +96,11 @@ type waitingState struct { // Represents who the request is waiting for. The conflicting // transaction may be a lock holder of a conflicting lock or a // conflicting request being sequenced through the same lockTable. - txn *enginepb.TxnMeta // always non-nil in waitFor{,Distinguished,Self} and waitElsewhere - key roachpb.Key // the key of the conflict - held bool // is the conflict a held lock? - queuedWriters int // how many writers are waiting? - queuedReaders int // how many readers are waiting? + txn *enginepb.TxnMeta // always non-nil in waitFor{,Distinguished,Self} and waitElsewhere + key roachpb.Key // the key of the conflict + held bool // is the conflict a held lock? + queuedLockingRequests int // how many locking requests are waiting? + queuedReaders int // how many readers are waiting? // Represents the lock strength of the action that the request was trying to // perform when it hit the conflict. E.g. was it trying to perform a (possibly @@ -125,8 +125,8 @@ func (s waitingState) SafeFormat(w redact.SafePrinter, _ rune) { if !s.held { target = "running request" } - w.Printf("wait for%s txn %s %s @ key %s (queuedWriters: %d, queuedReaders: %d)", - distinguished, s.txn.Short(), target, s.key, s.queuedWriters, s.queuedReaders) + w.Printf("wait for%s txn %s %s @ key %s (queuedLockingRequests: %d, queuedReaders: %d)", + distinguished, s.txn.Short(), target, s.key, s.queuedLockingRequests, s.queuedReaders) case waitSelf: w.Printf("wait self @ key %s", s.key) case waitElsewhere: @@ -136,7 +136,7 @@ func (s waitingState) SafeFormat(w redact.SafePrinter, _ rune) { w.Printf("wait elsewhere for txn %s @ key %s", s.txn.Short(), s.key) case waitQueueMaxLengthExceeded: w.Printf("wait-queue maximum length exceeded @ key %s with length %d", - s.key, s.queuedWriters) + s.key, s.queuedLockingRequests) case doneWaiting: w.SafeString("done waiting") default: @@ -371,9 +371,9 @@ func (t *lockTableImpl) setMaxKeysLocked(maxKeysLocked int64) { // claiming a lock on claimantTxn(). // // - The waitQueueMaxLengthExceeded state is used to indicate that the request -// was rejected because it attempted to enter a lock wait-queue as a writer -// and found that the queue's length was already equal to or exceeding the -// request's configured maximum. +// was rejected because it attempted to enter a lock wait-queue as a locking +// request and found that the queue's length was already equal to or +// exceeding the request's configured maximum. // // - The doneWaiting state is used to indicate that the request should make // another call to ScanAndEnqueue() (that next call is more likely to return a @@ -452,10 +452,10 @@ type lockTableGuardImpl struct { state waitingState signal chan struct{} - // locks for which this request is in the list of queued{Readers,Writers}. - // For writers, this includes both active and inactive waiters. For readers, - // there's no such thing as inactive readers, so by definition the request - // must be an active waiter. + // locks for which this request is in the list of + // queued{Readers,LockingRequests}. For locking requests, this includes both + // active and inactive waiters. For readers, there's no such thing as an + // inactive reader, so by definition the request must be an active waiter. // // TODO(sbhola): investigate whether the logic to maintain this locks map // can be simplified so it doesn't need to be adjusted by various keyLocks @@ -725,19 +725,19 @@ func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn( // There's no conflict with the lock holder itself. However, there may be // other locking requests that came before us that we may conflict with. - // Checking for conflicts with the list of queuedWriters ensures fairness by - // preventing a stream of locking[1] SKIP LOCKED requests from starving out - // regular locking requests. + // Checking for conflicts with the list of queuedLockingRequests ensures + // fairness by preventing a stream of locking[1] SKIP LOCKED requests from + // starving out regular locking requests. if str == lock.None { // [1] we only need to do this checking for locking requests return false, nil, nil } - for e := l.queuedWriters.Front(); e != nil; e = e.Next() { + for e := l.queuedLockingRequests.Front(); e != nil; e = e.Next() { qqg := e.Value.(*queuedGuard) 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 - // queuedWriters is sorted in increasing order of sequence number. + // queuedLockingRequests is sorted in increasing order of sequence number. break } if g.isSameTxn(qqg.guard.txnMeta()) { @@ -938,14 +938,15 @@ func (g *lockTableGuardImpl) resumeScan(notify bool) { } } -// Waiting writers in the keyLocks struct are wrapped in a queuedGuard. A -// waiting writer is typically waiting in an active state, i.e., the -// lockTableGuardImpl.key refers to this keyLocks struct. However, there are -// multiple reasons that can cause a writer to be an inactive waiter: -// - The first transactional writer is able to claim a lock when it is -// released. Doing so entails the writer being marked inactive. +// queuedGuard is used to wrap waiting locking requests in the keyLocks struct. +// Waiting requests typically wait in an active state, i.e., the +// lockTableGuardImpl.key refers to the same key inside this keyLock struct. +// However, there are multiple reasons that can cause a locking request to wait +// inactively at a keyLock: +// - The first locking request is able to claim a lock when it is released. +// Doing so entails the request being marked inactive. // - It is able to claim a lock that was previously claimed by a request with -// a higher sequence number. In such cases, the writer adds itself to the +// a higher sequence number. In such cases, the request adds itself to the // head of the queue as an inactive waiter and proceeds with its scan. // - A discovered lock causes the discoverer to become an inactive waiter // (until it scans again). @@ -953,8 +954,8 @@ func (g *lockTableGuardImpl) resumeScan(notify bool) { // waiter. // // The first two cases above (claiming an unheld lock) only occur for -// transactional requests, but the other cases can happen for both transactional -// and non-transactional requests. +// (transactional) locking requests, but the other cases can happen for +// both locking requests and non-transactional writers. type queuedGuard struct { guard *lockTableGuardImpl mode lock.Mode // protected by keyLocks.mu @@ -1218,7 +1219,7 @@ type txnLock struct { // Note that if the lock is replicated in nature, the lock would not be // tracked by the lock table when it was acquired as we do not track // uncontended replicated locks. Instead, this field would be initialized when - // a contending reader/writer discovered this lock. + // a contending request discovered this lock. startTime time.Time } @@ -1461,12 +1462,12 @@ type lockWaitQueue struct { // for cases that find a particular guard the lockTableGuardImpl.locks can be // a map instead of a set to point directly to the *list.Element. // - // queuedWriters: - // - to find all active queuedWriters. - // - to find the first active writer to make it distinguished. + // queuedLockingRequests: + // - to find all active queued locking requests. + // - to find the first active locking request to make it distinguished. // - to find a particular guard. // - to find the position, based on seqNum, for inserting a particular guard. - // - to find all waiting writers with a particular txn ID. + // - to find all waiting locking requests with a particular txn ID. // // waitingReaders: // - readers with a higher timestamp than some timestamp. @@ -1477,14 +1478,14 @@ type lockWaitQueue struct { // List of *queueGuard. The list is maintained in increasing order of sequence // numbers. This helps ensure some degree of fairness as requests are released - // from the head of the queue. Typically, this happens when the associated - // lock is released. + // from the head of the queue. Typically, this happens when all locks on the + // associated key are released. // // When a lock is not held, the head of the list should be comprised of an - // inactive, transactional writer (if the list is non-empty). Keeping its - // position as an inactive waiter at the head of the queue serves as a claim - // to prevent other concurrent requests (with higher sequence numbers) from - // barging in front of it. This is important for two reasons: + // inactive, transactional locking request (if the list is non-empty). Keeping + // its position as an inactive waiter at the head of the queue serves as a + // claim to prevent other concurrent locking requests (with higher sequence + // numbers) from barging in front of it. This is important for two reasons: // // 1. It helps ensure some degree of fairness, as sequence numbers are a proxy // for arrival order. @@ -1493,48 +1494,55 @@ type lockWaitQueue struct { // upfront to the lock table (as uncontended, replicated locks are only // discovered during evaluation). This means that no total ordering of lock // acquisition is enforced by the lock table -- using sequence numbers to - // break ties allows us to prevent deadlocks that would have arisen otherwise. + // break ties allows us to prevent deadlocks that could have arisen otherwise. // // Conversely, a request with a lower sequence number is allowed to barge in // front of an inactive waiter with a higher sequence number if the lock is // not held. This can be thought of as "breaking the claim" that the higher - // sequence numbered request tried to claim. As both these requests sequence - // through the lock table one of them will win the race. This is fine, as the - // request that wins the race can only evaluate while holding latches and the - // two requests must conflict on latches. As a result they're guaranteed to be - // isolated. We don't concern ourselves with the possible fairness issue if - // the higher sequence number wins the race. + // sequence numbered request tried to establish. As both these requests + // sequence through the lock table one of them will win the race. This is + // fine, as the request that wins the race can only evaluate while holding + // latches and the two requests must conflict on latches. As a result they're + // guaranteed to be isolated. We don't concern ourselves with the possible + // fairness issue if the higher sequence number wins the race. // // Non-locking readers are held in a separate list to the list of - // waitingReaders, and they make no claims on unheld locks like writers do. - // They race with the transactional writer that has made the claim. + // waitingReaders, and they make no claims on unheld locks like locking + // requests do. They race with the transactional locking requests that have + // made the claim. // // Similarly, non-transactional requests make no claims either, regardless of - // their read/write status. Non-transactional writes wait in the queuedWriters - // list along with transactional writers. The difference is as follows: + // their read/write status. Non-transactional writers wait in the + // queuedLockingRequests list along with transactional locking requests. The + // difference between non-transactional writers and transactional locking + // requests is as follows: + // // 1. When a lock transitions from held to released, the head of the queue - // that is made of non-transactional writes is cleared in one swoop (until we - // hit the first transactional writer or the queue is entirely drained). This - // means non-transactional writers race with a transactional writer's claim, - // like read requests. + // that is made up of non-transactional writes is cleared in one swoop (until + // we hit the first transactional writer or the queue is entirely drained). + // This means non-transactional writers race with transactional locking + // requests' claim(s), like read requests. // 2. When deciding whether to wait at an unheld lock or not, a // non-transactional writer will check how its sequence number compares to the - // head of the queuedWriters list. If its lower, it'll proceed; otherwise, + // head of the queuedLockingRequests list. If it's lower, it'll proceed + // without inserting itself into the queuedLockingRequests list; otherwise, // it'll wait. // // Multiple requests from the same transaction wait independently, including // the situation where one of the requests is an inactive waiter at the head // of the queue. However, if the inactive waiter manages to sequence, // evaluate, and acquire the lock, other requests from the same transaction - // are allowed to be released. + // with lock strength <= the strength with which the lock was acquired are + // allowed to proceed. // - // The behavior of only one transactional writer being allowed to make a claim - // by marking itself as inactive when a lock transitions from held to free is - // subject to change. As we introduce support for multiple locking strengths, - // and in particular locking strengths that are compatible with each other - // (read: shared locks), one could imagine a scheme where the head of the - // queuedWriters (s/queuedWriters/queuedLockers/g) that is compatible with - // each other is marked as inactive and allowed to proceed. A "joint claim". + // The behavior of only one transactional locking request being allowed to + // make a claim by marking itself as inactive when a lock transitions from + // held to free is subject to change. As we introduce support for multiple + // locking strengths, and in particular locking strengths that are compatible + // with each other (read: shared locks), one could imagine a scheme where the + // head of the queuedLockingRequests that is compatible with each other is + // marked as inactive and allowed to proceed. A "joint claim". + // TODO(arul): update the comment above, this change is coming soon! // // Once we introduce joint claims, we'll also need to support partially // breaking such claims. This means that a request that was previously @@ -1566,7 +1574,7 @@ type lockWaitQueue struct { // C req2 req3 // This is a deadlock caused by the lock table unless req2 partially // breaks the claim at A. - queuedWriters list.List + queuedLockingRequests list.List // List of *lockTableGuardImpl. All of these are actively waiting. If // non-empty, the lock must be held. By definition these cannot be in @@ -1675,9 +1683,9 @@ func (kl *keyLocks) safeFormat(sb *redact.StringBuilder, txnStatusCache *txnStat } } } - if kl.queuedWriters.Len() > 0 { - sb.SafeString(" queued writers:\n") - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + if kl.queuedLockingRequests.Len() > 0 { + sb.SafeString(" queued locking requests:\n") + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qg := e.Value.(*queuedGuard) g := qg.guard sb.Printf(" active: %t req: %d, strength: %s, txn: ", @@ -1706,21 +1714,21 @@ func (kl *keyLocks) collectLockStateInfo( defer kl.mu.Unlock() // Don't include locks that have neither lock holders, nor claims, nor - // waiting readers/writers. + // waiting readers/locking requests. if kl.isEmptyLock() { return false, roachpb.LockStateInfo{} } - // Filter out locks without waiting readers/writers unless explicitly + // Filter out locks without waiting readers/locking requests unless explicitly // requested. // // TODO(arul): This should consider the active/inactive status of all queued - // writers. If all waiting writers are inactive (and there are no waiting - // readers either), we should consider the lock to be uncontended. + // locking requests. If all waiting requests are inactive (and there are no + // waiting readers either), we should consider the lock to be uncontended. // See https://github.com/cockroachdb/cockroach/issues/103894. if !includeUncontended && kl.waitingReaders.Len() == 0 && - (kl.queuedWriters.Len() == 0 || - (kl.queuedWriters.Len() == 1 && !kl.queuedWriters.Front().Value.(*queuedGuard).active)) { + (kl.queuedLockingRequests.Len() == 0 || + (kl.queuedLockingRequests.Len() == 1 && !kl.queuedLockingRequests.Front().Value.(*queuedGuard).active)) { return false, roachpb.LockStateInfo{} } @@ -1743,10 +1751,10 @@ func (kl *keyLocks) lockStateInfo(now time.Time) roachpb.LockStateInfo { } } - waiterCount := kl.waitingReaders.Len() + kl.queuedWriters.Len() + waiterCount := kl.waitingReaders.Len() + kl.queuedLockingRequests.Len() lockWaiters := make([]lock.Waiter, 0, waiterCount) - // Add waiting readers before writers as they should run first. + // Add waiting readers before locking requests as they should run first. for e := kl.waitingReaders.Front(); e != nil; e = e.Next() { readerGuard := e.Value.(*lockTableGuardImpl) readerGuard.mu.Lock() @@ -1759,18 +1767,18 @@ func (kl *keyLocks) lockStateInfo(now time.Time) roachpb.LockStateInfo { readerGuard.mu.Unlock() } - // Lastly, add queued writers in order. - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + // Lastly, add queued locking requests, in order. + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qg := e.Value.(*queuedGuard) - writerGuard := qg.guard - writerGuard.mu.Lock() + g := qg.guard + g.mu.Lock() lockWaiters = append(lockWaiters, lock.Waiter{ - WaitingTxn: writerGuard.txnMeta(), + WaitingTxn: g.txnMeta(), ActiveWaiter: qg.active, Strength: lock.Exclusive, - WaitDuration: now.Sub(writerGuard.mu.curLockWaitStart), + WaitDuration: now.Sub(g.mu.curLockWaitStart), }) - writerGuard.mu.Unlock() + g.mu.Unlock() } return roachpb.LockStateInfo{ @@ -1795,7 +1803,7 @@ func (kl *keyLocks) addToMetrics(m *LockTableMetrics, now time.Time) { Held: kl.isLocked(), HoldDurationNanos: kl.lockHeldDuration(now).Nanoseconds(), WaitingReaders: int64(kl.waitingReaders.Len()), - WaitingWriters: int64(kl.queuedWriters.Len()), + WaitingWriters: int64(kl.queuedLockingRequests.Len()), WaitDurationNanos: totalWaitDuration.Nanoseconds(), MaxWaitDurationNanos: maxWaitDuration.Nanoseconds(), } @@ -1810,14 +1818,14 @@ func (kl *keyLocks) addToMetrics(m *LockTableMetrics, now time.Time) { // // REQUIRES: l.mu is locked. func (kl *keyLocks) informActiveWaiters() { - if kl.waitingReaders.Len() == 0 && kl.queuedWriters.Len() == 0 { + if kl.waitingReaders.Len() == 0 && kl.queuedLockingRequests.Len() == 0 { return // no active waiters to speak of; early return } waitForState := waitingState{ - kind: waitFor, - key: kl.key, - queuedWriters: kl.queuedWriters.Len(), - queuedReaders: kl.waitingReaders.Len(), + kind: waitFor, + key: kl.key, + queuedLockingRequests: kl.queuedLockingRequests.Len(), + queuedReaders: kl.waitingReaders.Len(), } // TODO(arul): This is entirely busted once we have multiple lock holders. // In such cases, there may be a request waiting not on the head of the @@ -1838,7 +1846,7 @@ func (kl *keyLocks) informActiveWaiters() { // NB: Note that if the distinguishedWaiter belongs to the same transaction // that waiters in the lock wait queue are waiting on, then the lock cannot // be held by it. This is because if it were, this request would no longer - // be waiting in lock wait queues (via a call to releaseWritersFromTxn). + // be waiting in lock wait queues (via a call to releaseLockingRequestsFromTxn). // This is asserted below. kl.distinguishedWaiter.isSameTxn(waitForState.txn) { // Ensure that if we're trying to find a new distinguished waiter because @@ -1858,7 +1866,8 @@ func (kl *keyLocks) informActiveWaiters() { state := waitForState // 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. + // 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.(*lockTableGuardImpl) if findDistinguished { @@ -1876,7 +1885,7 @@ func (kl *keyLocks) informActiveWaiters() { g.maybeUpdateWaitingStateLocked(state, true /* notify */) g.mu.Unlock() } - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qg := e.Value.(*queuedGuard) if !qg.active { continue @@ -1885,7 +1894,7 @@ func (kl *keyLocks) informActiveWaiters() { state := waitForState if g.isSameTxn(waitForState.txn) { if waitForState.held { - panic("writer from the lock holder txn should not be waiting in a wait queue") + panic("request from the lock holder txn should not be waiting in a wait queue") } state.kind = waitSelf } else { @@ -1938,25 +1947,25 @@ func (kl *keyLocks) claimantTxn() (_ *enginepb.TxnMeta, held bool) { // key. return kl.holders.Front().Value.(*txnLock).txn, true } - if kl.queuedWriters.Len() == 0 { - panic("no queued writers or lock holder; no one should be waiting on the lock") + if kl.queuedLockingRequests.Len() == 0 { + panic("no queued locking request or lock holder; no one should be waiting on the lock") } - qg := kl.queuedWriters.Front().Value.(*queuedGuard) + qg := kl.queuedLockingRequests.Front().Value.(*queuedGuard) return qg.guard.txnMeta(), false } -// releaseWritersFromTxn removes all waiting writers for the key, referenced in -// the receiver, that are part of the specified transaction. +// releaseLockingRequestsFromTxn removes all locking requests waiting on the +// key, referenced in the receiver, that are part of the specified transaction. // -// REQUIRES: l.mu is locked. -func (kl *keyLocks) releaseWritersFromTxn(txn *enginepb.TxnMeta) { - for e := kl.queuedWriters.Front(); e != nil; { +// REQUIRES: kl.mu is locked. +func (kl *keyLocks) releaseLockingRequestsFromTxn(txn *enginepb.TxnMeta) { + for e := kl.queuedLockingRequests.Front(); e != nil; { qg := e.Value.(*queuedGuard) curr := e e = e.Next() g := qg.guard if g.isSameTxn(txn) { - kl.removeWriter(curr) + kl.removeLockingRequest(curr) } } } @@ -1977,8 +1986,8 @@ func (kl *keyLocks) tryMakeNewDistinguished() { claimantTxn, _ := kl.claimantTxn() if kl.waitingReaders.Len() > 0 { g = kl.waitingReaders.Front().Value.(*lockTableGuardImpl) - } else if kl.queuedWriters.Len() > 0 { - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + } else if kl.queuedLockingRequests.Len() > 0 { + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qg := e.Value.(*queuedGuard) // Only requests actively waiting at this lock should be considered for // the distinguished distinction. @@ -2016,8 +2025,8 @@ func (kl *keyLocks) isEmptyLock() bool { assert(len(kl.heldBy) == 0, "non-empty heldBy map for an unlocked key") assert(kl.waitingReaders.Len() == 0, "keyLocks with waiting readers for unlocked key") // Determine if the lock is empty or not by checking the list of queued - // writers. - return kl.queuedWriters.Len() == 0 + // locking requests. + return kl.queuedLockingRequests.Len() == 0 } // assertEmptyLock asserts that the keyLocks is empty. This condition must hold @@ -2075,8 +2084,10 @@ func (kl *keyLocks) lockHeldDuration(now time.Time) time.Duration { } // Returns the total amount of time all waiters in the queues of -// readers and writers have been waiting on the lock. -// REQUIRES: l.mu is locked. +// readers and locking requests have been waiting on the key referenced in the +// reciever. +// +// REQUIRES: kl.mu is locked. func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time.Duration) { var totalWaitDuration time.Duration var maxWaitDuration time.Duration @@ -2090,7 +2101,7 @@ func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time. } g.mu.Unlock() } - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qg := e.Value.(*queuedGuard) g := qg.guard g.mu.Lock() @@ -2248,11 +2259,11 @@ func (kl *keyLocks) scanAndMaybeEnqueue(g *lockTableGuardImpl, notify bool) (wai // REQUIRES: l.mu to be locked. func (kl *keyLocks) constructWaitingState(g *lockTableGuardImpl) waitingState { waitForState := waitingState{ - kind: waitFor, - key: kl.key, - queuedWriters: kl.queuedWriters.Len(), - queuedReaders: kl.waitingReaders.Len(), - held: true, + kind: waitFor, + key: kl.key, + queuedLockingRequests: kl.queuedLockingRequests.Len(), + queuedReaders: kl.waitingReaders.Len(), + held: true, } txn, held := kl.claimantTxn() waitForState.held = held @@ -2295,12 +2306,13 @@ func (kl *keyLocks) alreadyHoldsLockAndIsAllowedToProceed( // TODO(arul): We want to allow requests that are writing to keys that they // hold exclusive locks on to "jump ahead" of any potential waiters. This // prevents deadlocks. The logic here is a bandaid until we implement a - // solution for the general case of arbitrary lock upgrades - // (e.g. shared -> exclusive, etc.). We'll do so by prioritizing requests - // from transaction's that hold locks over transactions that don't when - // storing them in the list of queuedWriters. Instead of sorting the list - // of queuedWriters just based on sequence numbers alone, we'll instead use - // (belongsToALockHolderTxn, sequence number) to construct the sort order. + // solution for the general case of arbitrary lock upgrades (e.g. shared -> + // exclusive, etc.). We'll do so by prioritizing requests from transaction's + // that hold locks over transactions that don't when storing them in the + // list of queuedLockingRequests. Instead of sorting the list of + // queuedLockingRequests just based on sequence numbers alone, we'll instead + // use (belongsToALockHolderTxn, sequence number) to construct the sort + // order. (str == lock.Intent && heldMode.Strength == lock.Exclusive) } @@ -2394,7 +2406,7 @@ func (kl *keyLocks) conflictsWithLockHolders(g *lockTableGuardImpl) bool { // // NB: If we were to not defer to the stronger lock strength and start // waiting here, we could potentially end up doing so in the wrong wait - // queue (queuedReaders vs. queuedWriters). There wouldn't be a + // queue (queuedReaders vs. queuedLockingRequests). There wouldn't be a // correctness issue in doing so, but it isn't ideal. // // NB: Non-transactional requests do not make claims or acquire locks. @@ -2467,7 +2479,7 @@ func (kl *keyLocks) enqueueLockingRequest(g *lockTableGuardImpl) (maxQueueLength // an inactive waiter at this lock and comes back around. if _, inQueue := g.mu.locks[kl]; inQueue { // Find the request; it must already be in the correct position. - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qqg := e.Value.(*queuedGuard) if qqg.guard == g { qqg.active = true // set the active status as true, in case it wasn't before @@ -2482,10 +2494,10 @@ func (kl *keyLocks) enqueueLockingRequest(g *lockTableGuardImpl) (maxQueueLength } // Check if the lock's wait queue has room for one more request. - if g.maxWaitQueueLength > 0 && kl.queuedWriters.Len() >= g.maxWaitQueueLength { + if g.maxWaitQueueLength > 0 && kl.queuedLockingRequests.Len() >= g.maxWaitQueueLength { // The wait-queue is longer than the request is willing to wait for. // Instead of entering the queue, immediately reject the request. For - // simplicity, we are not finding the position of this writer in the + // simplicity, we are not finding the position of this request in the // queue and rejecting the tail of the queue above the max length. That // would be more fair, but more complicated, and we expect that the // common case is that this waiter will be at the end of the queue. @@ -2499,16 +2511,16 @@ func (kl *keyLocks) enqueueLockingRequest(g *lockTableGuardImpl) (maxQueueLength // The request isn't in the queue. Add it in the correct position, based on // its sequence number. var e *list.Element - for e = kl.queuedWriters.Back(); e != nil; e = e.Prev() { + for e = kl.queuedLockingRequests.Back(); e != nil; e = e.Prev() { qqg := e.Value.(*queuedGuard) if qqg.guard.seqNum < qg.guard.seqNum { break } } if e == nil { - kl.queuedWriters.PushFront(qg) + kl.queuedLockingRequests.PushFront(qg) } else { - kl.queuedWriters.InsertAfter(qg, e) + kl.queuedLockingRequests.InsertAfter(qg, e) } // This request may be a candidate to become a distinguished waiter if one // doesn't exist yet; try making it such. @@ -2567,7 +2579,7 @@ func (kl *keyLocks) shouldRequestActivelyWait(g *lockTableGuardImpl) bool { // not support UPDATE locks. Even if it did, there would be no correctness // issue with what we're doing here, as long as the queue is maintained in // sequence number order. - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qqg := e.Value.(*queuedGuard) if qqg.guard == g { // We found our request while scanning from the front without finding any @@ -2592,10 +2604,10 @@ func (kl *keyLocks) shouldRequestActivelyWait(g *lockTableGuardImpl) bool { // responsibility to detect this case and actually choose one. Typically, this // is done using a call to informActiveWaiters. // -// [1] Only transactional, locking requests can lay claims. Non-transactional -// writers cannot. -// [2] While non-transactional writers cannot lay claims, they do need to be -// removed from the receiver's wait queue before proceeding. We do that here. +// [1] Only transactional, locking requests can establish claims. +// Non-transactional writers cannot. +// [2] While non-transactional writers cannot establish claims, they do need to +// be removed from the receiver's wait queue before proceeding. We do that here. // // REQUIRES: l.mu to be locked. func (kl *keyLocks) claimBeforeProceeding(g *lockTableGuardImpl) { @@ -2619,7 +2631,7 @@ func (kl *keyLocks) claimBeforeProceeding(g *lockTableGuardImpl) { // Find the request; iterate from the front, as requests proceeding are more // likely to be closer to the front than the back. - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qqg := e.Value.(*queuedGuard) if qqg.guard == g { // If the request was previously marked as a distinguished waiter, and is @@ -2634,9 +2646,9 @@ func (kl *keyLocks) claimBeforeProceeding(g *lockTableGuardImpl) { g.mu.Lock() delete(g.mu.locks, kl) g.mu.Unlock() - kl.queuedWriters.Remove(e) + kl.queuedLockingRequests.Remove(e) } else { - // Transactional writer. + // Transactional locking request. qqg.active = false // claim the lock } return @@ -2748,30 +2760,23 @@ func (kl *keyLocks) acquireLock(acq *roachpb.LockAcquisition, clock *hlc.Clock) return nil } - // NB: The lock isn't held, so the request trying to acquire the lock must be - // an (inactive) queued writer in the lock's wait queues. Typically, we expect - // this to be the first queued writer; the list of queued writers is - // maintained in lock table arrival order. When a lock transitions from held - // to released, the first of these writers is marked as inactive and allowed - // to proceed. This is done to uphold fairness between concurrent lock - // acquirers. However, in some rare cases[1], this may not be true -- i.e., - // the request trying to acquire the lock here may not be the first queued - // writer. This does not violate any correctness properties. This is because - // the request must be holding latches, as it has proceeded to evaluation for - // it to be calling into this method. As such, it is isolated from the first - // inactive queued writer. + // NB: The key isn't locked, so the request trying to acquire the lock must + // be an (inactive) request in the receiver's queuedLockingRequests wait + // queue. Typically, the request has a (possibly joint) claim on the key. + // However, in some rare cases[1], this may not be true -- i.e. the request + // may have had its claim broken by a different request being sequenced + // through the lock table. This does not violate any correctness properties, + // because the request must be holding latches to have proceeded to evaluation + // and finally ended up here. That means this request is isolated from the + // request that broke its claim by latching. // - // [1] Requests that run into conflicting locks drop their latches and enter - // its wait queues. Once the lock is released, and they can proceed with their - // scan, they do so without re-acquiring latches. In such cases, latches are - // acquired before evaluation. So they may insert themselves in front of - // another inactive waiting writer (which may or may not hold latches) if - // their arrival order dictates as such. The rare cases being talked about - // above are when the inactive waiting writer (in front of which the request - // inserted itself) was evaluating while holding latches and calls into this - // function once it finishes evaluation to actually acquire the lock. - - kl.releaseWritersFromTxn(&acq.Txn) + // [1] Requests that run into locks drop their latches and enter its wait + // queues. Once the lock is released, and they can proceed with their scan, + // they do so without re-acquiring latches. So it is possible for requests to + // break claims of requests that hold latches without holding latches + // themselves. + + kl.releaseLockingRequestsFromTxn(&acq.Txn) // Sanity check that there aren't any waiting readers on this lock. There // shouldn't be any, as the lock wasn't held. @@ -2850,7 +2855,7 @@ func (kl *keyLocks) discoveredLock( } case lock.Intent, lock.Exclusive: - // Immediately enter the lock's queuedWriters list. + // Immediately enter the lock's queuedLockingRequests list. // NB: this inactive waiter can be non-transactional. g.mu.Lock() _, presentHere := g.mu.locks[kl] @@ -2869,16 +2874,16 @@ func (kl *keyLocks) discoveredLock( } // g is not necessarily first in the queue in the (rare) case (a) above. var e *list.Element - for e = kl.queuedWriters.Front(); e != nil; e = e.Next() { + for e = kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qqg := e.Value.(*queuedGuard) if qqg.guard.seqNum > g.seqNum { break } } if e == nil { - kl.queuedWriters.PushBack(qg) + kl.queuedLockingRequests.PushBack(qg) } else { - kl.queuedWriters.InsertBefore(qg, e) + kl.queuedLockingRequests.InsertBefore(qg, e) } } default: @@ -2886,7 +2891,7 @@ func (kl *keyLocks) discoveredLock( } // If there are waiting requests from the same txn, they no longer need to wait. - kl.releaseWritersFromTxn(txn) + kl.releaseLockingRequestsFromTxn(txn) // Active waiters need to be told about who they are waiting for. kl.informActiveWaiters() @@ -2948,13 +2953,13 @@ func (kl *keyLocks) tryClearLock(force bool) bool { g.mu.Unlock() } - // Clear queuedWriters. - for e := kl.queuedWriters.Front(); e != nil; { + // Clear queuedLockingRequests. + for e := kl.queuedLockingRequests.Front(); e != nil; { qg := e.Value.(*queuedGuard) curr := e e = e.Next() - kl.queuedWriters.Remove(curr) + kl.queuedLockingRequests.Remove(curr) g := qg.guard g.mu.Lock() @@ -3121,13 +3126,16 @@ func (kl *keyLocks) increasedLockTs(newTs hlc.Timestamp) { } } -// removeWriter removes the writer, referenced by the supplied list.Element, -// from the lock's queuedWriters list. Returns whether the writer was the -// distinguished waiter. -func (kl *keyLocks) removeWriter(e *list.Element) bool { +// removeLockingRequest removes the locking request (or non-transactional +// writer), referenced by the supplied list.Element, from the lock's +// queuedLockingRequests list. Returns whether the request was the distinguished +// waiter or not. +// +// REQUIRES: kl.mu to be locked. +func (kl *keyLocks) removeLockingRequest(e *list.Element) bool { qg := e.Value.(*queuedGuard) g := qg.guard - kl.queuedWriters.Remove(e) + kl.queuedLockingRequests.Remove(e) g.mu.Lock() defer g.mu.Unlock() delete(g.mu.locks, kl) @@ -3159,11 +3167,11 @@ func (kl *keyLocks) removeReader(e *list.Element) bool { return false } -// A request known to the receiver is done. The request could be a waiting -// reader or writer. Note that there is the possibility of a race and the g may -// no longer be known to l, which we treat as a noop (this race is allowed since -// we order l.mu > g.mu). Returns whether the keyLocks struct can be garbage -// collected. +// A request known to the receiver is done. The request could be a locking or +// non-locking request. Note that there is the possibility of a race and the g +// may no longer be known to l, which we treat as a noop (this race is allowed +// since we order l.mu > g.mu). Returns whether the keyLocks struct can be +// garbage collected. // // Acquires l.mu. func (kl *keyLocks) requestDone(g *lockTableGuardImpl) (gc bool) { @@ -3178,13 +3186,13 @@ func (kl *keyLocks) requestDone(g *lockTableGuardImpl) (gc bool) { delete(g.mu.locks, kl) g.mu.Unlock() - // May be in queuedWriters or waitingReaders. + // May be in queuedLockingRequests or waitingReaders. distinguishedRemoved := false doneRemoval := false - for e := kl.queuedWriters.Front(); e != nil; e = e.Next() { + for e := kl.queuedLockingRequests.Front(); e != nil; e = e.Next() { qg := e.Value.(*queuedGuard) if qg.guard == g { - kl.queuedWriters.Remove(e) + kl.queuedLockingRequests.Remove(e) if qg.guard == kl.distinguishedWaiter { distinguishedRemoved = true kl.distinguishedWaiter = nil @@ -3195,12 +3203,12 @@ func (kl *keyLocks) requestDone(g *lockTableGuardImpl) (gc bool) { } if !kl.isLocked() && doneRemoval { - // The head of the list of waiting writers should always be an inactive, - // transactional writer if the lock isn't held. That may no longer be true - // if the guy we removed above was serving this purpose; the call to - // maybeReleaseFirstTransactionalWriter should fix that. And if it wasn't, - // it'll be a no-op. - kl.maybeReleaseFirstTransactionalWriter() + // The first request in the queuedLockingRequests should always be an + // inactive, transactional locking request if the lock isn't held. That may + // no longer be true if the guy we removed above was serving this purpose; + // the call to maybeReleaseFirstLockingRequest should fix that. And if + // it wasn't, it'll be a no-op. + kl.maybeReleaseFirstLockingRequest() } if !doneRemoval { @@ -3230,9 +3238,9 @@ func (kl *keyLocks) requestDone(g *lockTableGuardImpl) (gc bool) { // during the state transition from the Unreplicated durability to the // Replicated durability. This is possible because a Replicated lock is also // stored as an MVCC intent, so it does not need to also be stored in the -// lockTable if writers are not queuing on it. This is beneficial because it -// serves as a mitigation for #49973. Since we aren't currently great at -// avoiding excessive contention on limited scans when locks are in the +// lockTable if locking requests are not queuing on it. This is beneficial +// because it serves as a mitigation for #49973. Since we aren't currently great +// at avoiding excessive contention on limited scans when locks are in the // lockTable, it's better the keep locks out of the lockTable when possible. // // If any of the readers do truly contend with this lock even after their limit @@ -3251,13 +3259,13 @@ func (kl *keyLocks) tryFreeLockOnReplicatedAcquire() bool { return false } - // Bail if the lock has waiting writers. It is not uncontended. - if kl.queuedWriters.Len() != 0 { + // Bail if the lock has waiting locking requests. It is not uncontended. + if kl.queuedLockingRequests.Len() != 0 { return false } - // The lock is uncontended by other writers, so we're safe to drop it. - // This may release readers who were waiting on the lock. + // The lock is uncontended by other locking requests, so we're safe to drop + // it. This may release non-locking readers who were waiting on the lock. // // TODO(arul): Once we support replicated shared locks, we only want to clear // the lock holder that's promoting its durability from unreplicated to @@ -3272,11 +3280,11 @@ func (kl *keyLocks) tryFreeLockOnReplicatedAcquire() bool { // releaseWaitersOnKeyUnlocked is called when the key, referenced in the // receiver, transitions from locked to unlocked to handle state transitions for -// waiting{Readers,Writers}; if this results in there no longer being waiters on -// this key (read: the receiver is empty), a boolean gc=true is returned, -// indicating the receiver can be GC-ed by the caller. +// waitingReaders and queuedLockingRequests; if this results in there no longer +// being waiters on this key (read: the receiver is empty), a boolean gc=true is +// returned, indicating the receiver can be GC-ed by the caller. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) releaseWaitersOnKeyUnlocked() (gc bool) { assert(!kl.isLocked(), "releaseWaitersOnKeyUnlocked should only be called on unheld locks") @@ -3288,37 +3296,32 @@ func (kl *keyLocks) releaseWaitersOnKeyUnlocked() (gc bool) { kl.removeReader(curr) } - kl.maybeReleaseFirstTransactionalWriter() + kl.maybeReleaseFirstLockingRequest() // We've already cleared waiting readers above. The lock can be released if - // there are no waiting writers, active or otherwise. - if kl.queuedWriters.Len() == 0 { + // there are no waiting locking requests, active or otherwise. + if kl.queuedLockingRequests.Len() == 0 { kl.assertEmptyLock() return true } return false } -// maybeReleaseFirstTransactionalWriter goes through the list of writers waiting -// in the receiver's lock wait queues and, if present and actively waiting, -// releases the first transactional writer it finds. Releasing a transactional -// writer entails marking it as inactive and nudging it via a call to notify(). -// -// Any non-transactional writers at the head of the queue are also released. If -// no transactional writers are to be found, the list of queued writers will be -// empty when this function returns[1]. The function will no-op if the first -// transactional writer is already marked inactive (i.e. there's no releasing to -// do). +// maybeReleaseFirstLockingRequest goes through the list of locking requests waiting +// in the receiver's wait queues and, if present and actively waiting, +// releases the first locking request it finds. Releasing a locking request +// entails marking it as inactive and nudging it with a call to notify(). // -// [1] As if there are any requests in the list, they must be non-transactional, -// and all of them will be released. +// Any non-transactional writers at the head of the queue are also released. The +// function will no-op if the first locking request is already marked inactive +// (i.e. there's no releasing to do). // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. // REQUIRES: the (receiver) lock must not be held. // REQUIRES: there should not be any waitingReaders in the lock's wait queues. -func (kl *keyLocks) maybeReleaseFirstTransactionalWriter() { +func (kl *keyLocks) maybeReleaseFirstLockingRequest() { if kl.isLocked() { - panic("maybeReleaseFirstTransactionalWriter called when lock is held") + panic("maybeReleaseFirstLockingRequest called when lock is held") } if kl.waitingReaders.Len() != 0 { panic("there cannot be waiting readers") @@ -3326,25 +3329,25 @@ func (kl *keyLocks) maybeReleaseFirstTransactionalWriter() { // The prefix of the queue that is non-transactional writers is done // waiting. - for e := kl.queuedWriters.Front(); e != nil; { + for e := kl.queuedLockingRequests.Front(); e != nil; { qg := e.Value.(*queuedGuard) g := qg.guard - if g.txn != nil { // transactional writer + if g.txn != nil { // (transactional) locking request break } curr := e e = e.Next() - kl.removeWriter(curr) + kl.removeLockingRequest(curr) } - if kl.queuedWriters.Len() == 0 { - return // no transactional writer + if kl.queuedLockingRequests.Len() == 0 { + return // no locking requests } - // Check if the first (transactional) writer is active, and if it is, mark - // it as inactive. The call to doneActivelyWaitingAtLock should nudge it to - // pick up its scan from where it left off. - e := kl.queuedWriters.Front() + // Check if the first locking request is active, and if it is, mark it as + // inactive. The call to doneActivelyWaitingAtLock should nudge it to pick up + // its scan from where it left off. + e := kl.queuedLockingRequests.Front() qg := e.Value.(*queuedGuard) g := qg.guard if qg.active { @@ -3577,7 +3580,7 @@ func (t *lockTableImpl) AddDiscoveredLock( var lockSeqNum uint64 lockSeqNum, checkMaxLocks = t.locks.nextLockSeqNum() l = &keyLocks{id: lockSeqNum, key: key} - l.queuedWriters.Init() + l.queuedLockingRequests.Init() l.waitingReaders.Init() l.holders.Init() l.heldBy = make(map[uuid.UUID]*list.Element) @@ -3646,7 +3649,7 @@ func (t *lockTableImpl) AcquireLock(acq *roachpb.LockAcquisition) error { var lockSeqNum uint64 lockSeqNum, checkMaxLocks = t.locks.nextLockSeqNum() l = &keyLocks{id: lockSeqNum, key: acq.Key} - l.queuedWriters.Init() + l.queuedLockingRequests.Init() l.waitingReaders.Init() l.holders.Init() l.heldBy = make(map[uuid.UUID]*list.Element) diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index fba17b5e4b6c..f5b0792c9e43 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -1941,13 +1941,13 @@ func TestCanElideWaitingStateUpdateConsidersAllFields(t *testing.T) { doNotIncludeWhenDeciding inclusionStatus = false ) fieldMap := map[string]inclusionStatus{ - "kind": includeWhenDeciding, - "txn": includeWhenDeciding, - "key": includeWhenDeciding, - "held": includeWhenDeciding, - "queuedWriters": doNotIncludeWhenDeciding, - "queuedReaders": doNotIncludeWhenDeciding, - "guardStrength": doNotIncludeWhenDeciding, + "kind": includeWhenDeciding, + "txn": includeWhenDeciding, + "key": includeWhenDeciding, + "held": includeWhenDeciding, + "queuedLockingRequests": doNotIncludeWhenDeciding, + "queuedReaders": doNotIncludeWhenDeciding, + "guardStrength": doNotIncludeWhenDeciding, } ws := waitingState{} typ := reflect.ValueOf(ws).Type() diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic index cb88a273d0e9..8b27cebee1d1 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic @@ -113,7 +113,7 @@ sequence req=req3 [2] sequence req3: acquiring latches [2] sequence req3: scanning lock table for conflicting locks [2] sequence req3: waiting in lock wait-queues -[2] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[2] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [2] sequence req3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req3: pushing timestamp of txn 00000002 above 14.000000000,1 [2] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -200,7 +200,7 @@ sequence req=req5 [2] sequence req5: acquiring latches [2] sequence req5: scanning lock table for conflicting locks [2] sequence req5: waiting in lock wait-queues -[2] sequence req5: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[2] sequence req5: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [2] sequence req5: pushing after 0s for: liveness detection = true, deadlock detection = false, timeout enforcement = false, priority enforcement = false [2] sequence req5: pushing timestamp of txn 00000002 above 14.000000000,1 [2] sequence req5: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -215,7 +215,7 @@ sequence req=req6 [3] sequence req6: acquiring latches [3] sequence req6: scanning lock table for conflicting locks [3] sequence req6: waiting in lock wait-queues -[3] sequence req6: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 2) +[3] sequence req6: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 2) [3] sequence req6: not pushing [3] sequence req6: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -259,7 +259,7 @@ finish req=req6 [-] finish req6: finishing request [4] sequence req7: scanning lock table for conflicting locks [4] sequence req7: waiting in lock wait-queues -[4] sequence req7: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req7: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req7: pushing after 0s for: liveness detection = true, deadlock detection = false, timeout enforcement = false, priority enforcement = false [4] sequence req7: pushing txn 00000002 to abort [4] sequence req7: blocked on select in concurrency_test.(*cluster).PushTransaction diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents index b1a46e839227..10d2e215c082 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents @@ -63,7 +63,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -165,7 +165,7 @@ debug-lock-table num=1 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 sequence req=req1 @@ -174,7 +174,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing txn 00000002 to abort [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -201,11 +201,11 @@ debug-lock-table ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "b" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [holder finalized: committed] - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 sequence req=req1 @@ -230,14 +230,14 @@ debug-lock-table ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "b" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "c" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [holder finalized: committed] - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 sequence req=req1 @@ -257,13 +257,13 @@ debug-lock-table ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "b" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "c" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 finish req=req1 @@ -352,7 +352,7 @@ sequence req=req1 [4] sequence req1: acquiring latches [4] sequence req1: scanning lock table for conflicting locks [4] sequence req1: waiting in lock wait-queues -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -450,15 +450,15 @@ debug-lock-table num=3 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "d" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "e" holder: txn: 00000005-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 sequence req=req1 @@ -467,7 +467,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing txn 00000003 to abort [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -477,16 +477,16 @@ debug-lock-table num=3 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 lock: "d" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "e" holder: txn: 00000005-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 new-request name=req3 txn=txn3 ts=10,1 @@ -536,16 +536,16 @@ num=5 holder: txn: 00000004-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 lock: "d" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "e" holder: txn: 00000005-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 new-request name=req2 txn=txn2 ts=11,1 @@ -559,7 +559,7 @@ sequence req=req2 [6] sequence req2: acquiring latches [6] sequence req2: scanning lock table for conflicting locks [6] sequence req2: waiting in lock wait-queues -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req2: pushing timestamp of txn 00000003 above 11.000000000,1 [6] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -576,16 +576,16 @@ num=5 holder: txn: 00000004-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 lock: "d" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "e" holder: txn: 00000005-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 debug-advance-clock ts=123 @@ -597,13 +597,13 @@ on-txn-updated txn=txn3 status=aborted ---- [-] update txn: aborting txn3 [3] sequence req1: resolving intent ‹"c"› for txn 00000003 with ABORTED status -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000005 holding lock @ key ‹"e"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000005 holding lock @ key ‹"e"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req1: conflicted with ‹00000003-0000-0000-0000-000000000000› on ‹"c"› for 123.000s [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing txn 00000005 to abort [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction [6] sequence req2: resolving intent ‹"a"› for txn 00000003 with ABORTED status -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"b"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000003-0000-0000-0000-000000000000› on ‹"a"› for 123.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req2: pushing timestamp of txn 00000004 above 11.000000000,1 @@ -618,15 +618,15 @@ num=4 req: 8, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 8 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "d" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "e" holder: txn: 00000005-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 @@ -651,14 +651,14 @@ debug-lock-table ---- num=3 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "d" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 lock: "e" holder: txn: 00000005-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table index 11f27c3ea29f..fd8d79f2edf3 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table @@ -42,7 +42,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks index f41eed90949b..ca14d3cb7934 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks @@ -111,7 +111,7 @@ sequence req=req1r [4] sequence req1r: acquiring latches [4] sequence req1r: scanning lock table for conflicting locks [4] sequence req1r: waiting in lock wait-queues -[4] sequence req1r: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1r: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1r: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req1r: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1r: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -122,7 +122,7 @@ sequence req=req2r [5] sequence req2r: acquiring latches [5] sequence req2r: scanning lock table for conflicting locks [5] sequence req2r: waiting in lock wait-queues -[5] sequence req2r: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedWriters: 0, queuedReaders: 1) +[5] sequence req2r: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedLockingRequests: 0, queuedReaders: 1) [5] sequence req2r: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req2r: pushing timestamp of txn 00000003 above 10.000000000,1 [5] sequence req2r: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -135,7 +135,7 @@ sequence req=req3r [6] sequence req3r: acquiring latches [6] sequence req3r: scanning lock table for conflicting locks [6] sequence req3r: waiting in lock wait-queues -[6] sequence req3r: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req3r: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req3r: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req3r: pushing timestamp of txn 00000001 above 10.000000000,1 [6] sequence req3r: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -325,7 +325,7 @@ sequence req=req4w [4] sequence req4w: acquiring latches [4] sequence req4w: scanning lock table for conflicting locks [4] sequence req4w: waiting in lock wait-queues -[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req4w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4w: pushing txn 00000001 to abort [4] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -336,7 +336,7 @@ sequence req=req1w2 [5] sequence req1w2: acquiring latches [5] sequence req1w2: scanning lock table for conflicting locks [5] sequence req1w2: waiting in lock wait-queues -[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 1, queuedReaders: 0) +[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 1, queuedReaders: 0) [5] sequence req1w2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req1w2: pushing txn 00000002 to abort [5] sequence req1w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -347,7 +347,7 @@ sequence req=req2w2 [6] sequence req2w2: acquiring latches [6] sequence req2w2: scanning lock table for conflicting locks [6] sequence req2w2: waiting in lock wait-queues -[6] sequence req2w2: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 0) +[6] sequence req2w2: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) [6] sequence req2w2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req2w2: pushing txn 00000003 to abort [6] sequence req2w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -360,7 +360,7 @@ sequence req=req3w2 [7] sequence req3w2: acquiring latches [7] sequence req3w2: scanning lock table for conflicting locks [7] sequence req3w2: waiting in lock wait-queues -[7] sequence req3w2: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"a"› (queuedWriters: 2, queuedReaders: 0) +[7] sequence req3w2: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"a"› (queuedLockingRequests: 2, queuedReaders: 0) [7] sequence req3w2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [7] sequence req3w2: pushing txn 00000001 to abort [7] sequence req3w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -371,18 +371,18 @@ debug-lock-table num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 10, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 active: true req: 13, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 10 lock: "b" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 11, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 11 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 12, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 12 @@ -400,7 +400,7 @@ on-txn-updated txn=txn1 status=aborted [5] sequence req1w2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [5] sequence req1w2: sequencing complete, returned error: TransactionAbortedError(ABORT_REASON_PUSHER_ABORTED) [7] sequence req3w2: resolving intent ‹"a"› for txn 00000001 with ABORTED status -[7] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"a"› (queuedWriters: 2, queuedReaders: 0) +[7] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"a"› (queuedLockingRequests: 2, queuedReaders: 0) [7] sequence req3w2: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"a"› for 0.000s [7] sequence req3w2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [7] sequence req3w2: pushing txn 00000004 to detect request deadlock @@ -554,7 +554,7 @@ sequence req=req4w [4] sequence req4w: acquiring latches [4] sequence req4w: scanning lock table for conflicting locks [4] sequence req4w: waiting in lock wait-queues -[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req4w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4w: pushing txn 00000002 to abort [4] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -563,7 +563,7 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [4] sequence req4w: resolving intent ‹"b"› for txn 00000002 with COMMITTED status -[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req4w: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [4] sequence req4w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4w: pushing txn 00000003 to abort @@ -575,11 +575,11 @@ num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "b" - queued writers: + queued locking requests: active: false req: 17, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 17, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 17 @@ -601,7 +601,7 @@ sequence req=req1w2 [5] sequence req1w2: acquiring latches [5] sequence req1w2: scanning lock table for conflicting locks [5] sequence req1w2: waiting in lock wait-queues -[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"b"› (queuedWriters: 2, queuedReaders: 0) +[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"b"› (queuedLockingRequests: 2, queuedReaders: 0) [5] sequence req1w2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req1w2: pushing txn 00000004 to detect request deadlock [5] sequence req1w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -614,7 +614,7 @@ sequence req=req3w2 [6] sequence req3w2: acquiring latches [6] sequence req3w2: scanning lock table for conflicting locks [6] sequence req3w2: waiting in lock wait-queues -[6] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedWriters: 1, queuedReaders: 0) +[6] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedLockingRequests: 1, queuedReaders: 0) [6] sequence req3w2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req3w2: pushing txn 00000001 to abort [6] sequence req3w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -625,17 +625,17 @@ debug-lock-table num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 19, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 19 lock: "b" - queued writers: + queued locking requests: active: false req: 17, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 active: true req: 18, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 18 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 17, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 17 @@ -790,7 +790,7 @@ sequence req=req4w [4] sequence req4w: acquiring latches [4] sequence req4w: scanning lock table for conflicting locks [4] sequence req4w: waiting in lock wait-queues -[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req4w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4w: pushing txn 00000002 to abort [4] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -799,7 +799,7 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [4] sequence req4w: resolving intent ‹"b"› for txn 00000002 with COMMITTED status -[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req4w: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [4] sequence req4w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4w: pushing txn 00000003 to abort @@ -811,11 +811,11 @@ num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "b" - queued writers: + queued locking requests: active: false req: 23, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 23, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 23 @@ -837,7 +837,7 @@ sequence req=req1w2 [5] sequence req1w2: acquiring latches [5] sequence req1w2: scanning lock table for conflicting locks [5] sequence req1w2: waiting in lock wait-queues -[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"b"› (queuedWriters: 2, queuedReaders: 0) +[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"b"› (queuedLockingRequests: 2, queuedReaders: 0) [5] sequence req1w2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req1w2: pushing txn 00000004 to detect request deadlock [5] sequence req1w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -850,7 +850,7 @@ sequence req=req3w2 [6] sequence req3w2: acquiring latches [6] sequence req3w2: scanning lock table for conflicting locks [6] sequence req3w2: waiting in lock wait-queues -[6] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedWriters: 1, queuedReaders: 0) +[6] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedLockingRequests: 1, queuedReaders: 0) [6] sequence req3w2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req3w2: pushing txn 00000001 to abort [6] sequence req3w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -861,17 +861,17 @@ debug-lock-table num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 25, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 25 lock: "b" - queued writers: + queued locking requests: active: false req: 23, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 active: true req: 24, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 24 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 23, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 23 @@ -1038,7 +1038,7 @@ sequence req=req5w [4] sequence req5w: acquiring latches [4] sequence req5w: scanning lock table for conflicting locks [4] sequence req5w: waiting in lock wait-queues -[4] sequence req5w: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req5w: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req5w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req5w: pushing txn 00000002 to abort [4] sequence req5w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -1049,7 +1049,7 @@ sequence req=req4w [5] sequence req4w: acquiring latches [5] sequence req4w: scanning lock table for conflicting locks [5] sequence req4w: waiting in lock wait-queues -[5] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedWriters: 1, queuedReaders: 0) +[5] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"a"› (queuedLockingRequests: 1, queuedReaders: 0) [5] sequence req4w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req4w: pushing txn 00000001 to abort [5] sequence req4w: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -1058,7 +1058,7 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [5] sequence req4w: resolving intent ‹"a"› for txn 00000001 with COMMITTED status -[5] sequence req4w: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 2, queuedReaders: 0) +[5] sequence req4w: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 2, queuedReaders: 0) [5] sequence req4w: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"a"› for 0.000s [5] sequence req4w: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req4w: pushing txn 00000002 to abort @@ -1068,13 +1068,13 @@ on-txn-updated txn=txn2 status=committed ---- [-] update txn: committing txn2 [4] sequence req5w: resolving intent ‹"b"› for txn 00000002 with COMMITTED status -[4] sequence req5w: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req5w: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req5w: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [4] sequence req5w: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req5w: pushing txn 00000003 to abort [4] sequence req5w: blocked on select in concurrency_test.(*cluster).PushTransaction [5] sequence req4w: resolving intent ‹"b"› for txn 00000002 with COMMITTED status -[5] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000005 running request @ key ‹"b"› (queuedWriters: 2, queuedReaders: 0) +[5] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000005 running request @ key ‹"b"› (queuedLockingRequests: 2, queuedReaders: 0) [5] sequence req4w: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [5] sequence req4w: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req4w: pushing txn 00000005 to detect request deadlock @@ -1084,16 +1084,16 @@ debug-lock-table ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 30, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 lock: "b" - queued writers: + queued locking requests: active: false req: 29, strength: Intent, txn: 00000005-0000-0000-0000-000000000000 active: true req: 30, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 30 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 29, strength: Intent, txn: 00000005-0000-0000-0000-000000000000 distinguished req: 29 @@ -1113,7 +1113,7 @@ sequence req=req3w2 [6] sequence req3w2: acquiring latches [6] sequence req3w2: scanning lock table for conflicting locks [6] sequence req3w2: waiting in lock wait-queues -[6] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"a"› (queuedWriters: 2, queuedReaders: 0) +[6] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"a"› (queuedLockingRequests: 2, queuedReaders: 0) [6] sequence req3w2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req3w2: pushing txn 00000004 to detect request deadlock [6] sequence req3w2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -1123,18 +1123,18 @@ debug-lock-table ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 30, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 active: true req: 31, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 31 lock: "b" - queued writers: + queued locking requests: active: false req: 29, strength: Intent, txn: 00000005-0000-0000-0000-000000000000 active: true req: 30, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 30 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 29, strength: Intent, txn: 00000005-0000-0000-0000-000000000000 distinguished req: 29 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race index b1d1447de91e..6229cbad5019 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race @@ -146,7 +146,7 @@ sequence req=req4 [5] sequence req4: acquiring latches [5] sequence req4: scanning lock table for conflicting locks [5] sequence req4: waiting in lock wait-queues -[5] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[5] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [5] sequence req4: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req4: pushing timestamp of txn 00000003 above 10.000000000,0 [5] sequence req4: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -172,7 +172,7 @@ sequence req=req2 [7] sequence req2: acquiring latches [7] sequence req2: scanning lock table for conflicting locks [7] sequence req2: waiting in lock wait-queues -[7] sequence req2: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 2) +[7] sequence req2: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 2) [7] sequence req2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [7] sequence req2: pushing timestamp of txn 00000003 above 10.000000000,0 [7] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock index b0d22ad057eb..3b32301571e3 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock @@ -37,7 +37,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing timestamp of txn 00000001 above 12.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/isolation_level b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/isolation_level index 7cc118396c6f..a0c43c40750f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/isolation_level +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/isolation_level @@ -269,7 +269,7 @@ sequence req=req5 [7] sequence req5: acquiring latches [7] sequence req5: scanning lock table for conflicting locks [7] sequence req5: waiting in lock wait-queues -[7] sequence req5: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal1"› (queuedWriters: 0, queuedReaders: 1) +[7] sequence req5: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal1"› (queuedLockingRequests: 0, queuedReaders: 1) [7] sequence req5: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [7] sequence req5: pushing timestamp of txn 00000001 above 10.000000000,1 [7] sequence req5: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -299,7 +299,7 @@ sequence req=req6 [8] sequence req6: acquiring latches [8] sequence req6: scanning lock table for conflicting locks [8] sequence req6: waiting in lock wait-queues -[8] sequence req6: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal2"› (queuedWriters: 0, queuedReaders: 1) +[8] sequence req6: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal2"› (queuedLockingRequests: 0, queuedReaders: 1) [8] sequence req6: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [8] sequence req6: pushing timestamp of txn 00000001 above 11.000000000,1 [8] sequence req6: pusher pushed pushee to 11.000000000,2 @@ -336,7 +336,7 @@ sequence req=req7 [9] sequence req7: acquiring latches [9] sequence req7: scanning lock table for conflicting locks [9] sequence req7: waiting in lock wait-queues -[9] sequence req7: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal3"› (queuedWriters: 0, queuedReaders: 1) +[9] sequence req7: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal3"› (queuedLockingRequests: 0, queuedReaders: 1) [9] sequence req7: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [9] sequence req7: pushing timestamp of txn 00000001 above 12.000000000,1 [9] sequence req7: pusher pushed pushee to 12.000000000,2 @@ -369,7 +369,7 @@ sequence req=req8 [10] sequence req8: acquiring latches [10] sequence req8: scanning lock table for conflicting locks [10] sequence req8: waiting in lock wait-queues -[10] sequence req8: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal4"› (queuedWriters: 0, queuedReaders: 1) +[10] sequence req8: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kSSINormal4"› (queuedLockingRequests: 0, queuedReaders: 1) [10] sequence req8: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [10] sequence req8: pushing timestamp of txn 00000001 above 13.000000000,1 [10] sequence req8: pusher pushed pushee to 13.000000000,2 @@ -432,7 +432,7 @@ sequence req=req9 [11] sequence req9: acquiring latches [11] sequence req9: scanning lock table for conflicting locks [11] sequence req9: waiting in lock wait-queues -[11] sequence req9: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh1"› (queuedWriters: 0, queuedReaders: 1) +[11] sequence req9: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh1"› (queuedLockingRequests: 0, queuedReaders: 1) [11] sequence req9: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [11] sequence req9: pushing timestamp of txn 00000002 above 10.000000000,1 [11] sequence req9: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -455,7 +455,7 @@ sequence req=req10 [12] sequence req10: acquiring latches [12] sequence req10: scanning lock table for conflicting locks [12] sequence req10: waiting in lock wait-queues -[12] sequence req10: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh2"› (queuedWriters: 0, queuedReaders: 1) +[12] sequence req10: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh2"› (queuedLockingRequests: 0, queuedReaders: 1) [12] sequence req10: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [12] sequence req10: pushing timestamp of txn 00000002 above 11.000000000,1 [12] sequence req10: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -478,7 +478,7 @@ sequence req=req11 [13] sequence req11: acquiring latches [13] sequence req11: scanning lock table for conflicting locks [13] sequence req11: waiting in lock wait-queues -[13] sequence req11: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh3"› (queuedWriters: 0, queuedReaders: 1) +[13] sequence req11: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh3"› (queuedLockingRequests: 0, queuedReaders: 1) [13] sequence req11: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [13] sequence req11: pushing timestamp of txn 00000002 above 12.000000000,1 [13] sequence req11: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -520,7 +520,7 @@ sequence req=req12 [14] sequence req12: acquiring latches [14] sequence req12: scanning lock table for conflicting locks [14] sequence req12: waiting in lock wait-queues -[14] sequence req12: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh4"› (queuedWriters: 0, queuedReaders: 1) +[14] sequence req12: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kSSIHigh4"› (queuedLockingRequests: 0, queuedReaders: 1) [14] sequence req12: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [14] sequence req12: pushing timestamp of txn 00000002 above 13.000000000,1 [14] sequence req12: pusher pushed pushee to 13.000000000,2 @@ -584,7 +584,7 @@ sequence req=req13 [15] sequence req13: acquiring latches [15] sequence req13: scanning lock table for conflicting locks [15] sequence req13: waiting in lock wait-queues -[15] sequence req13: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal1"› (queuedWriters: 0, queuedReaders: 1) +[15] sequence req13: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal1"› (queuedLockingRequests: 0, queuedReaders: 1) [15] sequence req13: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [15] sequence req13: pushing timestamp of txn 00000003 above 10.000000000,1 [15] sequence req13: pusher pushed pushee to 10.000000000,2 @@ -616,7 +616,7 @@ sequence req=req14 [16] sequence req14: acquiring latches [16] sequence req14: scanning lock table for conflicting locks [16] sequence req14: waiting in lock wait-queues -[16] sequence req14: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal2"› (queuedWriters: 0, queuedReaders: 1) +[16] sequence req14: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal2"› (queuedLockingRequests: 0, queuedReaders: 1) [16] sequence req14: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [16] sequence req14: pushing timestamp of txn 00000003 above 11.000000000,1 [16] sequence req14: pusher pushed pushee to 11.000000000,2 @@ -648,7 +648,7 @@ sequence req=req15 [17] sequence req15: acquiring latches [17] sequence req15: scanning lock table for conflicting locks [17] sequence req15: waiting in lock wait-queues -[17] sequence req15: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal3"› (queuedWriters: 0, queuedReaders: 1) +[17] sequence req15: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal3"› (queuedLockingRequests: 0, queuedReaders: 1) [17] sequence req15: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [17] sequence req15: pushing timestamp of txn 00000003 above 12.000000000,1 [17] sequence req15: pusher pushed pushee to 12.000000000,2 @@ -680,7 +680,7 @@ sequence req=req16 [18] sequence req16: acquiring latches [18] sequence req16: scanning lock table for conflicting locks [18] sequence req16: waiting in lock wait-queues -[18] sequence req16: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal4"› (queuedWriters: 0, queuedReaders: 1) +[18] sequence req16: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kRCNormal4"› (queuedLockingRequests: 0, queuedReaders: 1) [18] sequence req16: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [18] sequence req16: pushing timestamp of txn 00000003 above 13.000000000,1 [18] sequence req16: pusher pushed pushee to 13.000000000,2 @@ -724,7 +724,7 @@ sequence req=req17 [19] sequence req17: acquiring latches [19] sequence req17: scanning lock table for conflicting locks [19] sequence req17: waiting in lock wait-queues -[19] sequence req17: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh1"› (queuedWriters: 0, queuedReaders: 1) +[19] sequence req17: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh1"› (queuedLockingRequests: 0, queuedReaders: 1) [19] sequence req17: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [19] sequence req17: pushing timestamp of txn 00000004 above 10.000000000,1 [19] sequence req17: pusher pushed pushee to 10.000000000,2 @@ -756,7 +756,7 @@ sequence req=req18 [20] sequence req18: acquiring latches [20] sequence req18: scanning lock table for conflicting locks [20] sequence req18: waiting in lock wait-queues -[20] sequence req18: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh2"› (queuedWriters: 0, queuedReaders: 1) +[20] sequence req18: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh2"› (queuedLockingRequests: 0, queuedReaders: 1) [20] sequence req18: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [20] sequence req18: pushing timestamp of txn 00000004 above 11.000000000,1 [20] sequence req18: pusher pushed pushee to 11.000000000,2 @@ -788,7 +788,7 @@ sequence req=req19 [21] sequence req19: acquiring latches [21] sequence req19: scanning lock table for conflicting locks [21] sequence req19: waiting in lock wait-queues -[21] sequence req19: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh3"› (queuedWriters: 0, queuedReaders: 1) +[21] sequence req19: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh3"› (queuedLockingRequests: 0, queuedReaders: 1) [21] sequence req19: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [21] sequence req19: pushing timestamp of txn 00000004 above 12.000000000,1 [21] sequence req19: pusher pushed pushee to 12.000000000,2 @@ -820,7 +820,7 @@ sequence req=req20 [22] sequence req20: acquiring latches [22] sequence req20: scanning lock table for conflicting locks [22] sequence req20: waiting in lock wait-queues -[22] sequence req20: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh4"› (queuedWriters: 0, queuedReaders: 1) +[22] sequence req20: lock wait-queue event: wait for (distinguished) txn 00000004 holding lock @ key ‹"kRCHigh4"› (queuedLockingRequests: 0, queuedReaders: 1) [22] sequence req20: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [22] sequence req20: pushing timestamp of txn 00000004 above 13.000000000,1 [22] sequence req20: pusher pushed pushee to 13.000000000,2 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout index 1cec3f497a70..aef4b55f3cbf 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout @@ -70,7 +70,7 @@ sequence req=req3 [3] sequence req3: acquiring latches [3] sequence req3: scanning lock table for conflicting locks [3] sequence req3: waiting in lock wait-queues -[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k2"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k2"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req3: pushing txn 00000001 to abort [3] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -82,7 +82,7 @@ num=3 holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] lock: "k2" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k3" @@ -103,7 +103,7 @@ sequence req=reqTimeout1 [4] sequence reqTimeout1: acquiring latches [4] sequence reqTimeout1: scanning lock table for conflicting locks [4] sequence reqTimeout1: waiting in lock wait-queues -[4] sequence reqTimeout1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence reqTimeout1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence reqTimeout1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = true, priority enforcement = false [4] sequence reqTimeout1: pushing txn 00000001 to check if abandoned [4] sequence reqTimeout1: pushee not abandoned @@ -119,7 +119,7 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [3] sequence req3: resolving intent ‹"k2"› for txn 00000001 with COMMITTED status -[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k3"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k3"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req3: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"k2"› for 0.000s [3] sequence req3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req3: pushing txn 00000002 to abort @@ -140,11 +140,11 @@ debug-lock-table ---- num=2 lock: "k2" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 @@ -164,7 +164,7 @@ sequence req=reqTimeout2 [6] sequence reqTimeout2: acquiring latches [6] sequence reqTimeout2: scanning lock table for conflicting locks [6] sequence reqTimeout2: waiting in lock wait-queues -[6] sequence reqTimeout2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k2"› (queuedWriters: 2, queuedReaders: 0) +[6] sequence reqTimeout2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k2"› (queuedLockingRequests: 2, queuedReaders: 0) [6] sequence reqTimeout2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = true, priority enforcement = false [6] sequence reqTimeout2: conflicted with ‹00000003-0000-0000-0000-000000000000› on ‹"k2"› for 0.000s [6] sequence reqTimeout2: sequencing complete, returned error: conflicting locks on ‹"k2"› [reason=lock_timeout] @@ -196,7 +196,7 @@ sequence req=reqTimeout3 [9] sequence reqTimeout3: acquiring latches [9] sequence reqTimeout3: scanning lock table for conflicting locks [9] sequence reqTimeout3: waiting in lock wait-queues -[9] sequence reqTimeout3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k4"› (queuedWriters: 0, queuedReaders: 1) +[9] sequence reqTimeout3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k4"› (queuedLockingRequests: 0, queuedReaders: 1) [9] sequence reqTimeout3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = true, priority enforcement = false [9] sequence reqTimeout3: pushing txn 00000002 to check if abandoned [9] sequence reqTimeout3: pushee not abandoned @@ -207,11 +207,11 @@ debug-lock-table ---- num=3 lock: "k2" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k4" diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic index c0d4c4a8ab77..f71755252dba 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic @@ -89,7 +89,7 @@ sequence req=req3 eval-kind=pess-after-opt [4] sequence req3: optimistic failed, so waiting for latches [4] sequence req3: scanning lock table for conflicting locks [4] sequence req3: waiting in lock wait-queues -[4] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"d"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"d"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req3: pushing timestamp of txn 00000001 above 12.000000000,1 [4] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority index 71efdc70fd2b..3e7ba32c1fa5 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority @@ -137,7 +137,7 @@ sequence req=req4 [4] sequence req4: acquiring latches [4] sequence req4: scanning lock table for conflicting locks [4] sequence req4: waiting in lock wait-queues -[4] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kLow1"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kLow1"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req4: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4: pushing timestamp of txn 00000001 above 10.000000000,1 [4] sequence req4: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -154,7 +154,7 @@ sequence req=req5 [5] sequence req5: acquiring latches [5] sequence req5: scanning lock table for conflicting locks [5] sequence req5: waiting in lock wait-queues -[5] sequence req5: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"kLow1"› (queuedWriters: 0, queuedReaders: 2) +[5] sequence req5: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"kLow1"› (queuedLockingRequests: 0, queuedReaders: 2) [5] sequence req5: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = true [5] sequence req5: pushing timestamp of txn 00000001 above 10.000000000,1 [5] sequence req5: pusher pushed pushee to 10.000000000,2 @@ -209,7 +209,7 @@ sequence req=req6 [6] sequence req6: acquiring latches [6] sequence req6: scanning lock table for conflicting locks [6] sequence req6: waiting in lock wait-queues -[6] sequence req6: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kLow2"› (queuedWriters: 1, queuedReaders: 0) +[6] sequence req6: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"kLow2"› (queuedLockingRequests: 1, queuedReaders: 0) [6] sequence req6: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [6] sequence req6: pushing txn 00000001 to abort [6] sequence req6: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -226,12 +226,12 @@ sequence req=req7 [7] sequence req7: acquiring latches [7] sequence req7: scanning lock table for conflicting locks [7] sequence req7: waiting in lock wait-queues -[7] sequence req7: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"kLow2"› (queuedWriters: 2, queuedReaders: 0) +[7] sequence req7: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"kLow2"› (queuedLockingRequests: 2, queuedReaders: 0) [7] sequence req7: pushing after 0s for: liveness detection = false, deadlock detection = false, timeout enforcement = false, priority enforcement = true [7] sequence req7: pushing txn 00000001 to abort [7] sequence req7: pusher aborted pushee [7] sequence req7: resolving intent ‹"kLow2"› for txn 00000001 with ABORTED status -[7] sequence req7: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"kLow2"› (queuedWriters: 2, queuedReaders: 0) +[7] sequence req7: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"kLow2"› (queuedLockingRequests: 2, queuedReaders: 0) [7] sequence req7: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"kLow2"› for 0.000s [7] sequence req7: pushing after 0s for: liveness detection = false, deadlock detection = false, timeout enforcement = false, priority enforcement = true [7] sequence req7: pushing txn 00000004 to detect request deadlock @@ -285,7 +285,7 @@ sequence req=req8 [8] sequence req8: acquiring latches [8] sequence req8: scanning lock table for conflicting locks [8] sequence req8: waiting in lock wait-queues -[8] sequence req8: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kNormal1"› (queuedWriters: 0, queuedReaders: 1) +[8] sequence req8: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kNormal1"› (queuedLockingRequests: 0, queuedReaders: 1) [8] sequence req8: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [8] sequence req8: pushing timestamp of txn 00000002 above 10.000000000,1 [8] sequence req8: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -302,7 +302,7 @@ sequence req=req9 [9] sequence req9: acquiring latches [9] sequence req9: scanning lock table for conflicting locks [9] sequence req9: waiting in lock wait-queues -[9] sequence req9: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"kNormal1"› (queuedWriters: 0, queuedReaders: 2) +[9] sequence req9: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"kNormal1"› (queuedLockingRequests: 0, queuedReaders: 2) [9] sequence req9: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = true [9] sequence req9: pushing timestamp of txn 00000002 above 10.000000000,1 [9] sequence req9: pusher pushed pushee to 10.000000000,2 @@ -355,7 +355,7 @@ sequence req=req10 [10] sequence req10: acquiring latches [10] sequence req10: scanning lock table for conflicting locks [10] sequence req10: waiting in lock wait-queues -[10] sequence req10: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kNormal2"› (queuedWriters: 1, queuedReaders: 0) +[10] sequence req10: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"kNormal2"› (queuedLockingRequests: 1, queuedReaders: 0) [10] sequence req10: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [10] sequence req10: pushing txn 00000002 to abort [10] sequence req10: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -372,12 +372,12 @@ sequence req=req11 [11] sequence req11: acquiring latches [11] sequence req11: scanning lock table for conflicting locks [11] sequence req11: waiting in lock wait-queues -[11] sequence req11: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"kNormal2"› (queuedWriters: 2, queuedReaders: 0) +[11] sequence req11: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"kNormal2"› (queuedLockingRequests: 2, queuedReaders: 0) [11] sequence req11: pushing after 0s for: liveness detection = false, deadlock detection = false, timeout enforcement = false, priority enforcement = true [11] sequence req11: pushing txn 00000002 to abort [11] sequence req11: pusher aborted pushee [11] sequence req11: resolving intent ‹"kNormal2"› for txn 00000002 with ABORTED status -[11] sequence req11: lock wait-queue event: wait for (distinguished) txn 00000007 running request @ key ‹"kNormal2"› (queuedWriters: 2, queuedReaders: 0) +[11] sequence req11: lock wait-queue event: wait for (distinguished) txn 00000007 running request @ key ‹"kNormal2"› (queuedLockingRequests: 2, queuedReaders: 0) [11] sequence req11: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"kNormal2"› for 0.000s [11] sequence req11: pushing after 0s for: liveness detection = false, deadlock detection = false, timeout enforcement = false, priority enforcement = true [11] sequence req11: pushing txn 00000007 to detect request deadlock @@ -429,7 +429,7 @@ sequence req=req12 [12] sequence req12: acquiring latches [12] sequence req12: scanning lock table for conflicting locks [12] sequence req12: waiting in lock wait-queues -[12] sequence req12: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kHigh1"› (queuedWriters: 0, queuedReaders: 1) +[12] sequence req12: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kHigh1"› (queuedLockingRequests: 0, queuedReaders: 1) [12] sequence req12: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [12] sequence req12: pushing timestamp of txn 00000003 above 10.000000000,1 [12] sequence req12: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -440,7 +440,7 @@ sequence req=req13 [13] sequence req13: acquiring latches [13] sequence req13: scanning lock table for conflicting locks [13] sequence req13: waiting in lock wait-queues -[13] sequence req13: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"kHigh1"› (queuedWriters: 0, queuedReaders: 2) +[13] sequence req13: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"kHigh1"› (queuedLockingRequests: 0, queuedReaders: 2) [13] sequence req13: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [13] sequence req13: pushing timestamp of txn 00000003 above 10.000000000,1 [13] sequence req13: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -501,7 +501,7 @@ sequence req=req14 [14] sequence req14: acquiring latches [14] sequence req14: scanning lock table for conflicting locks [14] sequence req14: waiting in lock wait-queues -[14] sequence req14: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kHigh2"› (queuedWriters: 1, queuedReaders: 0) +[14] sequence req14: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"kHigh2"› (queuedLockingRequests: 1, queuedReaders: 0) [14] sequence req14: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [14] sequence req14: pushing txn 00000003 to abort [14] sequence req14: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -512,7 +512,7 @@ sequence req=req15 [15] sequence req15: acquiring latches [15] sequence req15: scanning lock table for conflicting locks [15] sequence req15: waiting in lock wait-queues -[15] sequence req15: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"kHigh2"› (queuedWriters: 2, queuedReaders: 0) +[15] sequence req15: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"kHigh2"› (queuedLockingRequests: 2, queuedReaders: 0) [15] sequence req15: not pushing [15] sequence req15: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -525,7 +525,7 @@ on-txn-updated txn=txnHighPushee status=committed [14] sequence req14: acquiring latches [14] sequence req14: scanning lock table for conflicting locks [14] sequence req14: sequencing complete, returned guard -[15] sequence req15: lock wait-queue event: wait for (distinguished) txn 00000008 running request @ key ‹"kHigh2"› (queuedWriters: 2, queuedReaders: 0) +[15] sequence req15: lock wait-queue event: wait for (distinguished) txn 00000008 running request @ key ‹"kHigh2"› (queuedLockingRequests: 2, queuedReaders: 0) [15] sequence req15: conflicted with ‹00000003-0000-0000-0000-000000000000› on ‹"kHigh2"› for 0.000s [15] sequence req15: pushing after 0s for: liveness detection = false, deadlock detection = false, timeout enforcement = false, priority enforcement = true [15] sequence req15: pushing txn 00000008 to detect request deadlock diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded index fde7bbcff28f..fa50fea14b87 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded @@ -50,7 +50,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [2] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: pushing txn 00000001 to abort [2] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -65,7 +65,7 @@ sequence req=req3 [3] sequence req3: acquiring latches [3] sequence req3: scanning lock table for conflicting locks [3] sequence req3: waiting in lock wait-queues -[3] sequence req3: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 2, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 2, queuedReaders: 0) [3] sequence req3: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req3: pushing txn 00000001 to abort [3] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -80,7 +80,7 @@ sequence req=req4 [4] sequence req4: acquiring latches [4] sequence req4: scanning lock table for conflicting locks [4] sequence req4: waiting in lock wait-queues -[4] sequence req4: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 3, queuedReaders: 0) +[4] sequence req4: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 3, queuedReaders: 0) [4] sequence req4: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4: pushing txn 00000001 to abort [4] sequence req4: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -90,7 +90,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 active: true req: 4, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 @@ -112,7 +112,7 @@ sequence req=req5r [5] sequence req5r: acquiring latches [5] sequence req5r: scanning lock table for conflicting locks [5] sequence req5r: waiting in lock wait-queues -[5] sequence req5r: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 3, queuedReaders: 1) +[5] sequence req5r: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 3, queuedReaders: 1) [5] sequence req5r: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req5r: pushing timestamp of txn 00000001 above 10.000000000,1 [5] sequence req5r: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -127,13 +127,13 @@ on-txn-updated txn=txn1 status=committed [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: sequencing complete, returned guard [3] sequence req3: resolving intent ‹"k"› for txn 00000001 with COMMITTED status -[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 running request @ key ‹"k"› (queuedWriters: 3, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 running request @ key ‹"k"› (queuedLockingRequests: 3, queuedReaders: 0) [3] sequence req3: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"k"› for 0.000s [3] sequence req3: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req3: pushing txn 00000002 to detect request deadlock [3] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction [4] sequence req4: resolving intent ‹"k"› for txn 00000001 with COMMITTED status -[4] sequence req4: lock wait-queue event: wait for txn 00000002 running request @ key ‹"k"› (queuedWriters: 3, queuedReaders: 0) +[4] sequence req4: lock wait-queue event: wait for txn 00000002 running request @ key ‹"k"› (queuedLockingRequests: 3, queuedReaders: 0) [4] sequence req4: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"k"› for 0.000s [4] sequence req4: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4: pushing txn 00000002 to detect request deadlock @@ -152,11 +152,11 @@ finish req=req5r on-lock-acquired req=req2 key=k ---- [-] acquire lock: txn 00000002 @ ‹k› -[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 2, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 2, queuedReaders: 0) [3] sequence req3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req3: pushing txn 00000002 to abort [3] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction -[4] sequence req4: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 2, queuedReaders: 0) +[4] sequence req4: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 2, queuedReaders: 0) [4] sequence req4: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4: pushing txn 00000002 to abort [4] sequence req4: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -170,7 +170,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 active: true req: 4, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 3 @@ -207,7 +207,7 @@ on-txn-updated txn=txn2 status=aborted [3] sequence req3: scanning lock table for conflicting locks [3] sequence req3: sequencing complete, returned guard [4] sequence req4: resolving intent ‹"k"› for txn 00000002 with ABORTED status -[4] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k"› (queuedWriters: 2, queuedReaders: 0) +[4] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k"› (queuedLockingRequests: 2, queuedReaders: 0) [4] sequence req4: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"k"› for 0.000s [4] sequence req4: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4: pushing txn 00000003 to detect request deadlock diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener index e2f1f6bb23a3..d9cb2672724a 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener @@ -93,7 +93,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [2] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -102,7 +102,7 @@ debug-lock-table num=2 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 2 lock: "k2" @@ -165,7 +165,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 sequence req=req2 @@ -174,7 +174,7 @@ sequence req=req2 [7] sequence req2: acquiring latches [7] sequence req2: scanning lock table for conflicting locks [7] sequence req2: waiting in lock wait-queues -[7] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[7] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [7] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [7] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -212,7 +212,7 @@ debug-lock-table ---- num=1 lock: "k" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k @@ -280,7 +280,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 sequence req=req3 @@ -289,7 +289,7 @@ sequence req=req3 [13] sequence req3: acquiring latches [13] sequence req3: scanning lock table for conflicting locks [13] sequence req3: waiting in lock wait-queues -[13] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[13] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [13] sequence req3: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [13] sequence req3: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -322,7 +322,7 @@ debug-lock-table ---- num=1 lock: "k" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 on-lock-acquired req=req3 key=k @@ -402,7 +402,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [2] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -411,7 +411,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 6, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 6 @@ -438,7 +438,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 sequence req=req2 @@ -447,7 +447,7 @@ sequence req=req2 [4] sequence req2: acquiring latches [4] sequence req2: scanning lock table for conflicting locks [4] sequence req2: waiting in lock wait-queues -[4] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -480,7 +480,7 @@ debug-lock-table ---- num=1 lock: "k" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k @@ -574,7 +574,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [2] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -590,7 +590,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 8, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 8 @@ -658,7 +658,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 10, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 sequence req=req2 @@ -667,7 +667,7 @@ sequence req=req2 [8] sequence req2: acquiring latches [8] sequence req2: scanning lock table for conflicting locks [8] sequence req2: waiting in lock wait-queues -[8] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[8] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [8] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [8] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -700,7 +700,7 @@ debug-lock-table ---- num=1 lock: "k" - queued writers: + queued locking requests: active: false req: 10, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k @@ -780,7 +780,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [2] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -789,7 +789,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 12, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 12 @@ -816,7 +816,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 12, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 sequence req=req2 @@ -825,7 +825,7 @@ sequence req=req2 [4] sequence req2: acquiring latches [4] sequence req2: scanning lock table for conflicting locks [4] sequence req2: waiting in lock wait-queues -[4] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req2: pushing after 1h0m0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req2: blocked on select in concurrency.(*lockTableWaiterImpl).WaitOn @@ -858,7 +858,7 @@ debug-lock-table ---- num=1 lock: "k" - queued writers: + queued locking requests: active: false req: 12, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents index f3e46889e973..36ee90856e1e 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents @@ -73,7 +73,7 @@ sequence req=req2 [3] sequence req2: acquiring latches [3] sequence req2: scanning lock table for conflicting locks [3] sequence req2: waiting in lock wait-queues -[3] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req2: pushing txn 00000002 to abort [3] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -85,7 +85,7 @@ sequence req=req1 [4] sequence req1: acquiring latches [4] sequence req1: scanning lock table for conflicting locks [4] sequence req1: waiting in lock wait-queues -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: pusher pushed pushee to 10.000000000,2 @@ -115,7 +115,7 @@ debug-lock-table ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 on-txn-updated txn=txn2 status=aborted @@ -214,7 +214,7 @@ sequence req=req1 [4] sequence req1: acquiring latches [4] sequence req1: scanning lock table for conflicting locks [4] sequence req1: waiting in lock wait-queues -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: pusher pushed pushee to 10.000000000,2 @@ -288,17 +288,17 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [3] sequence req1: pushing timestamp of txn 00000002 above 11.000000000,1 [3] sequence req1: pusher pushed pushee to 11.000000000,2 [3] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,6} -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"a"› for 0.000s [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [3] sequence req1: pushing timestamp of txn 00000002 above 11.000000000,1 [3] sequence req1: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,8} -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [3] sequence req1: pushing timestamp of txn 00000002 above 11.000000000,1 @@ -399,7 +399,7 @@ sequence req=req2 [3] sequence req2: acquiring latches [3] sequence req2: scanning lock table for conflicting locks [3] sequence req2: waiting in lock wait-queues -[3] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req2: pushing txn 00000002 to abort [3] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -411,52 +411,52 @@ sequence req=req1 [4] sequence req1: acquiring latches [4] sequence req1: scanning lock table for conflicting locks [4] sequence req1: waiting in lock wait-queues -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: pusher pushed pushee to 10.000000000,2 [4] sequence req1: resolving intent ‹"a"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,13} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"a"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,15} -[4] sequence req1: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"c"› (queuedWriters: 1, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 1, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"c"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,17} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"d"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"d"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"d"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,19} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"e"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"e"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"d"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"e"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,21} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"f"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"f"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"e"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"f"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,23} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"g"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"g"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"f"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"g"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,25} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"h"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"h"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"g"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"h"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,27} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"i"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"i"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"h"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [4] sequence req1: resolving intent ‹"i"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,29} -[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"j"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"j"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"i"› for 0.000s [4] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 @@ -476,7 +476,7 @@ debug-lock-table ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 7, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 on-txn-updated txn=txn2 status=aborted diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table index 2e76784de096..e70d122cd305 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table @@ -42,7 +42,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [3] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [3] sequence req1: pusher pushed pushee to 10.000000000,2 @@ -154,7 +154,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [3] sequence req1: pushing timestamp of txn 00000002 above 11.000000000,1 [3] sequence req1: pusher pushed pushee to 11.000000000,2 @@ -226,46 +226,46 @@ sequence req=req2 [6] sequence req2: acquiring latches [6] sequence req2: scanning lock table for conflicting locks [6] sequence req2: waiting in lock wait-queues -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,5} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"c"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,7} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"d"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"d"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"d"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,9} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"e"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"e"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"d"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"e"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,11} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"f"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"f"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"e"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"f"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,13} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"g"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"g"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"f"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"g"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,15} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"h"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"h"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"g"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"h"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,17} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"i"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"i"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"h"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 [6] sequence req2: resolving intent ‹"i"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,19} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"j"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"j"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"i"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 11.000000000,1 @@ -325,7 +325,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"a"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [3] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 [3] sequence req1: pusher pushed pushee to 10.000000000,2 @@ -397,46 +397,46 @@ sequence req=req2 [6] sequence req2: acquiring latches [6] sequence req2: scanning lock table for conflicting locks [6] sequence req2: waiting in lock wait-queues -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"b"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"b"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,25} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"c"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"b"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"c"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,27} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"d"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"d"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"c"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"d"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,29} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"e"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"e"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"d"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"e"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,31} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"f"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"f"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"e"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"f"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,33} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"g"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"g"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"f"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"g"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,35} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"h"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"h"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"g"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"h"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,37} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"i"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"i"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"h"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 [6] sequence req2: resolving intent ‹"i"› for txn 00000002 with PENDING status and clock observation {1 123.000000000,39} -[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"j"› (queuedWriters: 0, queuedReaders: 1) +[6] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"j"› (queuedLockingRequests: 0, queuedReaders: 1) [6] sequence req2: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"i"› for 0.000s [6] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = true [6] sequence req2: pushing timestamp of txn 00000002 above 10.000000000,1 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty index 885b05f61694..b6be233a8817 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty @@ -39,7 +39,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing timestamp of txn 00000001 above 15.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -105,7 +105,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing timestamp of txn 00000001 above 135.000000000,0 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -174,7 +174,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing timestamp of txn 00000001 above 150.000000000,1? [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -249,7 +249,7 @@ sequence req=req1 [3] sequence req1: acquiring latches [3] sequence req1: scanning lock table for conflicting locks [3] sequence req1: waiting in lock wait-queues -[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[3] sequence req1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [3] sequence req1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req1: pushing timestamp of txn 00000001 above 15.000000000,1 [3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -288,7 +288,7 @@ sequence req=req2-retry [5] sequence req2-retry: acquiring latches [5] sequence req2-retry: scanning lock table for conflicting locks [5] sequence req2-retry: waiting in lock wait-queues -[5] sequence req2-retry: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 2) +[5] sequence req2-retry: lock wait-queue event: wait for txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 2) [5] sequence req2-retry: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [5] sequence req2-retry: pushing timestamp of txn 00000001 above 15.000000000,1 [5] sequence req2-retry: blocked on select in concurrency_test.(*cluster).PushTransaction diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update index 39b50581470b..5f65f2482952 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update @@ -49,7 +49,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [2] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: pushing timestamp of txn 00000001 above 12.000000000,1 [2] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -168,7 +168,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [2] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: pushing timestamp of txn 00000001 above 12.000000000,1 [2] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -296,7 +296,7 @@ sequence req=req2 [2] sequence req2: acquiring latches [2] sequence req2: scanning lock table for conflicting locks [2] sequence req2: waiting in lock wait-queues -[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[2] sequence req2: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [2] sequence req2: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence req2: pushing timestamp of txn 00000001 above 12.000000000,1 [2] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -353,7 +353,7 @@ sequence req=req4 [3] sequence req4: acquiring latches [3] sequence req4: scanning lock table for conflicting locks [3] sequence req4: waiting in lock wait-queues -[3] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req4: pushing after 0s for: liveness detection = true, deadlock detection = false, timeout enforcement = false, priority enforcement = false [3] sequence req4: pushing txn 00000001 to abort [3] sequence req4: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -382,7 +382,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl, unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 9, strength: Intent, txn: none distinguished req: 9 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere index 00fdd86b83f0..866f1429163c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere @@ -60,7 +60,7 @@ sequence req=reqWaiter [4] sequence reqWaiter: acquiring latches [4] sequence reqWaiter: scanning lock table for conflicting locks [4] sequence reqWaiter: waiting in lock wait-queues -[4] sequence reqWaiter: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence reqWaiter: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence reqWaiter: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence reqWaiter: pushing txn 00000001 to abort [4] sequence reqWaiter: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -70,7 +70,7 @@ debug-lock-table num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 2 @@ -202,7 +202,7 @@ sequence req=reqTwoKeyWaiter [10] sequence reqTwoKeyWaiter: acquiring latches [10] sequence reqTwoKeyWaiter: scanning lock table for conflicting locks [10] sequence reqTwoKeyWaiter: waiting in lock wait-queues -[10] sequence reqTwoKeyWaiter: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"k1"› (queuedWriters: 0, queuedReaders: 1) +[10] sequence reqTwoKeyWaiter: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"k1"› (queuedLockingRequests: 0, queuedReaders: 1) [10] sequence reqTwoKeyWaiter: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [10] sequence reqTwoKeyWaiter: pushing timestamp of txn 00000003 above 20.000000000,1 [10] sequence reqTwoKeyWaiter: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -233,7 +233,7 @@ sequence req=reqThreeKeyWaiter [12] sequence reqThreeKeyWaiter: acquiring latches [12] sequence reqThreeKeyWaiter: scanning lock table for conflicting locks [12] sequence reqThreeKeyWaiter: waiting in lock wait-queues -[12] sequence reqThreeKeyWaiter: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"k1"› (queuedWriters: 0, queuedReaders: 2) +[12] sequence reqThreeKeyWaiter: lock wait-queue event: wait for txn 00000003 holding lock @ key ‹"k1"› (queuedLockingRequests: 0, queuedReaders: 2) [12] sequence reqThreeKeyWaiter: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [12] sequence reqThreeKeyWaiter: pushing timestamp of txn 00000003 above 20.000000000,1 [12] sequence reqThreeKeyWaiter: blocked on select in concurrency_test.(*cluster).PushTransaction diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error index 76cf76f21039..acaeea5bbaa6 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error @@ -70,7 +70,7 @@ sequence req=req3 [3] sequence req3: acquiring latches [3] sequence req3: scanning lock table for conflicting locks [3] sequence req3: waiting in lock wait-queues -[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k2"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k2"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req3: pushing txn 00000001 to abort [3] sequence req3: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -82,7 +82,7 @@ num=3 holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] lock: "k2" holder: txn: 00000001-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k3" @@ -103,7 +103,7 @@ sequence req=reqNoWait1 [4] sequence reqNoWait1: acquiring latches [4] sequence reqNoWait1: scanning lock table for conflicting locks [4] sequence reqNoWait1: waiting in lock wait-queues -[4] sequence reqNoWait1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedWriters: 0, queuedReaders: 1) +[4] sequence reqNoWait1: lock wait-queue event: wait for (distinguished) txn 00000001 holding lock @ key ‹"k"› (queuedLockingRequests: 0, queuedReaders: 1) [4] sequence reqNoWait1: pushing txn 00000001 to check if abandoned [4] sequence reqNoWait1: pushee not abandoned [4] sequence reqNoWait1: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"k"› for 0.000s @@ -121,7 +121,7 @@ on-txn-updated txn=txn1 status=committed ---- [-] update txn: committing txn1 [3] sequence req3: resolving intent ‹"k2"› for txn 00000001 with COMMITTED status -[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k3"› (queuedWriters: 1, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k3"› (queuedLockingRequests: 1, queuedReaders: 0) [3] sequence req3: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"k2"› for 123.000s [3] sequence req3: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence req3: pushing txn 00000002 to abort @@ -142,11 +142,11 @@ debug-lock-table ---- num=2 lock: "k2" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 @@ -166,7 +166,7 @@ sequence req=reqNoWait2 [6] sequence reqNoWait2: acquiring latches [6] sequence reqNoWait2: scanning lock table for conflicting locks [6] sequence reqNoWait2: waiting in lock wait-queues -[6] sequence reqNoWait2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k2"› (queuedWriters: 2, queuedReaders: 0) +[6] sequence reqNoWait2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k2"› (queuedLockingRequests: 2, queuedReaders: 0) [6] sequence reqNoWait2: conflicted with ‹00000003-0000-0000-0000-000000000000› on ‹"k2"› for 0.000s [6] sequence reqNoWait2: sequencing complete, returned error: conflicting locks on ‹"k2"› [reason=wait_policy] @@ -197,7 +197,7 @@ sequence req=reqNoWait3 [9] sequence reqNoWait3: acquiring latches [9] sequence reqNoWait3: scanning lock table for conflicting locks [9] sequence reqNoWait3: waiting in lock wait-queues -[9] sequence reqNoWait3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k4"› (queuedWriters: 0, queuedReaders: 1) +[9] sequence reqNoWait3: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k4"› (queuedLockingRequests: 0, queuedReaders: 1) [9] sequence reqNoWait3: pushing txn 00000002 to check if abandoned [9] sequence reqNoWait3: pushee not abandoned [9] sequence reqNoWait3: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"k4"› for 0.000s @@ -207,11 +207,11 @@ debug-lock-table ---- num=3 lock: "k2" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k4" diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip index 68d5ba8ac375..c8d7e6086822 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip @@ -88,7 +88,7 @@ sequence req=req4 [4] sequence req4: acquiring latches [4] sequence req4: scanning lock table for conflicting locks [4] sequence req4: waiting in lock wait-queues -[4] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"k4"› (queuedWriters: 1, queuedReaders: 0) +[4] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000003 holding lock @ key ‹"k4"› (queuedLockingRequests: 1, queuedReaders: 0) [4] sequence req4: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence req4: pushing txn 00000003 to abort [4] sequence req4: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -113,7 +113,7 @@ num=4 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] lock: "k4" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000004-0000-0000-0000-000000000000 # ------------------------------------------------------------- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self index 9489c9f1e6ae..a0be42a9e3a4 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self @@ -47,7 +47,7 @@ sequence req=reqTxn1 [2] sequence reqTxn1: acquiring latches [2] sequence reqTxn1: scanning lock table for conflicting locks [2] sequence reqTxn1: waiting in lock wait-queues -[2] sequence reqTxn1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 1, queuedReaders: 0) +[2] sequence reqTxn1: lock wait-queue event: wait for (distinguished) txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 1, queuedReaders: 0) [2] sequence reqTxn1: pushing after 0s for: liveness detection = true, deadlock detection = true, timeout enforcement = false, priority enforcement = false [2] sequence reqTxn1: pushing txn 00000002 to abort [2] sequence reqTxn1: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -58,7 +58,7 @@ sequence req=reqTxnMiddle [3] sequence reqTxnMiddle: acquiring latches [3] sequence reqTxnMiddle: scanning lock table for conflicting locks [3] sequence reqTxnMiddle: waiting in lock wait-queues -[3] sequence reqTxnMiddle: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 2, queuedReaders: 0) +[3] sequence reqTxnMiddle: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 2, queuedReaders: 0) [3] sequence reqTxnMiddle: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence reqTxnMiddle: pushing txn 00000002 to abort [3] sequence reqTxnMiddle: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -69,7 +69,7 @@ sequence req=reqTxn2 [4] sequence reqTxn2: acquiring latches [4] sequence reqTxn2: scanning lock table for conflicting locks [4] sequence reqTxn2: waiting in lock wait-queues -[4] sequence reqTxn2: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedWriters: 3, queuedReaders: 0) +[4] sequence reqTxn2: lock wait-queue event: wait for txn 00000002 holding lock @ key ‹"k"› (queuedLockingRequests: 3, queuedReaders: 0) [4] sequence reqTxn2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence reqTxn2: pushing txn 00000002 to abort [4] sequence reqTxn2: blocked on select in concurrency_test.(*cluster).PushTransaction @@ -87,7 +87,7 @@ on-txn-updated txn=txnOld status=committed [2] sequence reqTxn1: scanning lock table for conflicting locks [2] sequence reqTxn1: sequencing complete, returned guard [3] sequence reqTxnMiddle: resolving intent ‹"k"› for txn 00000002 with COMMITTED status -[3] sequence reqTxnMiddle: lock wait-queue event: wait for (distinguished) txn 00000001 running request @ key ‹"k"› (queuedWriters: 3, queuedReaders: 0) +[3] sequence reqTxnMiddle: lock wait-queue event: wait for (distinguished) txn 00000001 running request @ key ‹"k"› (queuedLockingRequests: 3, queuedReaders: 0) [3] sequence reqTxnMiddle: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"k"› for 123.000s [3] sequence reqTxnMiddle: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [3] sequence reqTxnMiddle: pushing txn 00000001 to detect request deadlock @@ -101,7 +101,7 @@ debug-lock-table ---- num=1 lock: "k" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 active: true req: 3, strength: Intent, txn: 00000003-0000-0000-0000-000000000000 active: true req: 4, strength: Intent, txn: 00000001-0000-0000-0000-000000000000 @@ -121,7 +121,7 @@ finish req=reqTxn1 [3] sequence reqTxnMiddle: acquiring latches [3] sequence reqTxnMiddle: scanning lock table for conflicting locks [3] sequence reqTxnMiddle: sequencing complete, returned guard -[4] sequence reqTxn2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k"› (queuedWriters: 2, queuedReaders: 0) +[4] sequence reqTxn2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k"› (queuedLockingRequests: 2, queuedReaders: 0) [4] sequence reqTxn2: conflicted with ‹00000001-0000-0000-0000-000000000000› on ‹"k"› for 0.000s [4] sequence reqTxn2: pushing after 0s for: liveness detection = false, deadlock detection = true, timeout enforcement = false, priority enforcement = false [4] sequence reqTxn2: pushing txn 00000003 to detect request deadlock diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs index 68c47570525e..60ff3d31c70e 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs @@ -116,7 +116,7 @@ acquire r=req7 k=a durability=r ignored-seqs=8 strength=intent num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl, unrepl [(str: Exclusive seq: 1)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 @@ -130,7 +130,7 @@ acquire r=req8 k=a durability=u ignored-seqs=8 strength=exclusive num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl, unrepl [(str: Exclusive seq: 1)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 @@ -149,7 +149,7 @@ acquire r=req9 k=a durability=u ignored-seqs=1,5-9 strength=exclusive num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl, unrepl [(str: Exclusive seq: 11)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 @@ -171,6 +171,6 @@ acquire r=req10 k=a durability=r strength=intent num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered index 8718a61efb19..18d481aa07a7 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered @@ -68,7 +68,7 @@ release txn=txn1 span=a ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 @@ -90,7 +90,7 @@ add-discovered r=req2 k=a txn=txn3 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 scan r=req2 @@ -110,7 +110,7 @@ dequeue r=req3 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -118,7 +118,7 @@ release txn=txn3 span=a ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 guard-state r=req2 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic index df0b0d9d2864..9afcf528e809 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic @@ -134,7 +134,7 @@ update txn=txn1 ts=11,1 epoch=1 span=b ---- num=3 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -156,10 +156,10 @@ update txn=txn1 ts=11,1 epoch=1 span=c,e ---- num=3 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -187,13 +187,13 @@ add-discovered r=req4 k=a txn=txn3 num=4 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 6.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -203,13 +203,13 @@ add-discovered r=req4 k=f txn=txn3 num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 6.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -254,14 +254,14 @@ print num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 6.000000000,0, info: repl - queued writers: + queued locking requests: active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -392,14 +392,14 @@ dequeue r=req5 num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 6.000000000,0, info: repl - queued writers: + queued locking requests: active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -440,16 +440,16 @@ print num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 6.000000000,0, info: repl - queued writers: + queued locking requests: active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -595,16 +595,16 @@ print num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 6.000000000,0, info: repl - queued writers: + queued locking requests: active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -736,15 +736,15 @@ release txn=txn3 span=a ---- num=5 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -765,15 +765,15 @@ print ---- num=5 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -908,15 +908,15 @@ release txn=txn3 span=f ---- num=4 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -951,15 +951,15 @@ acquire r=req4 k=b durability=r strength=intent ---- num=4 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -970,16 +970,16 @@ acquire r=req4 k=c durability=r strength=intent ---- num=4 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -997,16 +997,16 @@ print ---- num=4 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -1113,12 +1113,12 @@ dequeue r=req4 num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -1149,11 +1149,11 @@ release txn=txn2 span=c,f num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -1171,11 +1171,11 @@ print num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl - queued writers: + queued locking requests: active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - queued writers: + queued locking requests: active: false req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -1285,10 +1285,10 @@ release txn=txn2 span=b ---- num=3 lock: "b" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -1321,10 +1321,10 @@ print ---- num=3 lock: "b" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: false req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "e" @@ -1439,7 +1439,7 @@ dequeue r=req6 ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] @@ -1736,7 +1736,7 @@ print num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 10, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 @@ -1836,7 +1836,7 @@ release txn=txn1 span=c ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 10, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 @@ -1862,7 +1862,7 @@ print ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 10, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 @@ -1962,7 +1962,7 @@ acquire r=req10 k=c durability=u strength=exclusive num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -1980,7 +1980,7 @@ print num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -2079,7 +2079,7 @@ dequeue r=req10 num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -2088,7 +2088,7 @@ acquire r=req12 k=c durability=r strength=intent num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl, unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -2097,7 +2097,7 @@ dequeue r=req12 num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 8.000000000,12, info: repl, unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -2109,7 +2109,7 @@ release txn=txn2 span=b,d ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req11 @@ -2120,7 +2120,7 @@ print ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 11, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 metrics @@ -2250,14 +2250,14 @@ release txn=txn2 span=b,d ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 14, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req15 ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 14, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 new-request r=req16 txn=none ts=10,12 spans=none@c @@ -2418,12 +2418,12 @@ print num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 9.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 18, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 18 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 9.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 @@ -2524,11 +2524,11 @@ release txn=txn1 span=c ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 18, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 9.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 @@ -2540,11 +2540,11 @@ print ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 18, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 9.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 18, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 @@ -2645,10 +2645,10 @@ release txn=txn1 span=d ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 18, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "d" - queued writers: + queued locking requests: active: false req: 18, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 19, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 @@ -2660,7 +2660,7 @@ acquire r=req18 k=d durability=u strength=exclusive ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 18, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] @@ -2746,7 +2746,7 @@ print num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 22, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 22 lock: "d" @@ -2855,11 +2855,11 @@ release txn=txn1 span=d num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 1, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 22, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 22 lock: "d" - queued writers: + queued locking requests: active: false req: 23, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req23 @@ -2879,10 +2879,10 @@ release txn=txn1 span=c ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 22, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "d" - queued writers: + queued locking requests: active: false req: 23, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req22 @@ -2893,10 +2893,10 @@ print ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 22, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "d" - queued writers: + queued locking requests: active: false req: 22, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: false req: 23, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 @@ -2995,11 +2995,11 @@ acquire r=req23 k=d durability=u strength=exclusive ---- num=2 lock: "c" - queued writers: + queued locking requests: active: false req: 22, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: false req: 22, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 dequeue r=req22 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear index 36ba57526083..13f2ce6a96ef 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear @@ -116,12 +116,12 @@ num=3 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] waiting readers: req: 3, txn: none - queued writers: + queued locking requests: active: true req: 4, strength: Intent, txn: none distinguished req: 3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks index 5c4399cbea30..4820312eb715 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks @@ -48,7 +48,7 @@ add-discovered r=req1 k=a txn=txn2 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 add-discovered r=req1 k=b txn=txn2 @@ -56,11 +56,11 @@ add-discovered r=req1 k=b txn=txn2 num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 add-discovered r=req1 k=d txn=txn3 @@ -68,15 +68,15 @@ add-discovered r=req1 k=d txn=txn3 num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 add-discovered r=req1 k=e txn=txn3 @@ -84,19 +84,19 @@ add-discovered r=req1 k=e txn=txn3 num=4 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 scan r=req2 @@ -108,21 +108,21 @@ acquire r=req2 k=c durability=u strength=exclusive num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000004 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req2 @@ -130,21 +130,21 @@ dequeue r=req2 num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000004 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 scan r=req1 @@ -156,22 +156,22 @@ print num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000004 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 pushed-txn-updated txn=txn2 status=aborted @@ -190,24 +190,24 @@ print num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000004 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 pushed-txn-updated txn=txn3 status=committed @@ -218,22 +218,22 @@ release txn=txn4 span=c num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl [holder finalized: committed] - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl [holder finalized: committed] - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 @@ -250,19 +250,19 @@ print ---- num=5 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "d" - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - queued writers: + queued locking requests: active: false req: 1, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req1 @@ -296,7 +296,7 @@ add-discovered r=req3 k=a txn=txn2 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 add-discovered r=req3 k=c txn=txn2 @@ -304,11 +304,11 @@ add-discovered r=req3 k=c txn=txn2 num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 scan r=req4 @@ -320,13 +320,13 @@ acquire r=req4 k=b durability=u strength=exclusive num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req4 @@ -334,13 +334,13 @@ dequeue r=req4 num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 pushed-txn-updated txn=txn2 status=aborted @@ -361,13 +361,13 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 3, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req3 @@ -399,7 +399,7 @@ add-discovered r=req5 k=a txn=txn2 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 add-discovered r=req5 k=b txn=txn2 @@ -407,11 +407,11 @@ add-discovered r=req5 k=b txn=txn2 num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 scan r=req5 @@ -423,12 +423,12 @@ print num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 5 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 pushed-txn-updated txn=txn2 status=aborted @@ -451,26 +451,26 @@ print num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: true req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 active: true req: 6, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 release txn=txn2 span=a ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 active: true req: 6, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 6 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 guard-state r=req6 @@ -487,19 +487,19 @@ print ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 active: true req: 6, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 6 lock: "b" - queued writers: + queued locking requests: active: false req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req5 ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req6 @@ -531,7 +531,7 @@ add-discovered r=req7 k=a txn=txn2 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 add-discovered r=req7 k=b txn=txn2 @@ -539,11 +539,11 @@ add-discovered r=req7 k=b txn=txn2 num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 scan r=req7 @@ -559,12 +559,12 @@ print num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 7 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 pushed-txn-updated txn=txn2 status=aborted @@ -589,10 +589,10 @@ print ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req7 @@ -734,7 +734,7 @@ add-discovered r=req11 k=a txn=txn2 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 12.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 11, strength: Intent, txn: none pushed-txn-updated txn=txn2 status=aborted @@ -918,10 +918,10 @@ print ---- num=6 lock: "a" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "b" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000006 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] [holder finalized: aborted] @@ -944,10 +944,10 @@ print ---- num=4 lock: "a" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "b" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000006 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] [holder finalized: aborted] @@ -958,10 +958,10 @@ print ---- num=4 lock: "a" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "b" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000006 epoch: 0, iso: Serializable, ts: 11.000000000,0, info: unrepl [(str: Exclusive seq: 0)] [holder finalized: aborted] @@ -980,10 +980,10 @@ print ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "b" - queued writers: + queued locking requests: active: false req: 13, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 # ----------------------------------------------------------------------------- @@ -1006,7 +1006,7 @@ add-discovered r=req17 k=a txn=txn7 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000007 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 add-discovered r=req17 k=b txn=txn8 @@ -1014,11 +1014,11 @@ add-discovered r=req17 k=b txn=txn8 num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000007 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000008 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 scan r=req17 @@ -1030,12 +1030,12 @@ print num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000007 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 distinguished req: 16 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000008 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 pushed-txn-updated txn=txn7 status=aborted @@ -1056,11 +1056,11 @@ print num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000007 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000008 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 distinguished req: 16 @@ -1077,12 +1077,12 @@ print num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000007 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 active: true req: 17, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000008 distinguished req: 17 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000008 epoch: 0, iso: Serializable, ts: 11.000000000,1, info: repl - queued writers: + queued locking requests: active: true req: 16, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000005 distinguished req: 16 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable index 3e3cb69b1126..48a00e4b28ea 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable @@ -74,7 +74,7 @@ add-discovered r=req2 k=a txn=txn2 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: repl - queued writers: + queued locking requests: active: false req: 2, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 scan r=req2 @@ -89,7 +89,7 @@ release txn=txn2 span=a ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 guard-state r=req2 @@ -100,7 +100,7 @@ acquire r=req2 k=c durability=u strength=exclusive ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache b/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache index cfe253fb7e5c..ed9635e4d0d6 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache @@ -41,7 +41,7 @@ add-discovered r=req1 k=a txn=txn2 consult-txn-status-cache=false num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl [holder finalized: aborted] - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 # Nothing to resolve yet. @@ -63,7 +63,7 @@ print ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 scan r=req1 @@ -78,7 +78,7 @@ add-discovered r=req1 k=b txn=txn3 consult-txn-status-cache=true ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 # Txn is finalized and txnStatusCache is consulted. @@ -86,7 +86,7 @@ add-discovered r=req1 k=c txn=txn3 consult-txn-status-cache=true ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 # Txn is not finalized and txnStatusCache is consulted. @@ -94,11 +94,11 @@ add-discovered r=req1 k=d txn=txn4 consult-txn-status-cache=true ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000004 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 # Locks for b and c were not added to lock table. diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access index 4f78d0ac6ec2..b3906e92218c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access @@ -52,7 +52,7 @@ print 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -60,7 +60,7 @@ release txn=txn1 span=a ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 guard-state r=req2 @@ -144,12 +144,12 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" @@ -162,11 +162,11 @@ release txn=txn1 span=b num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] @@ -180,15 +180,15 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -198,14 +198,14 @@ release txn=txn1 span=a ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -217,15 +217,15 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -235,14 +235,14 @@ release txn=txn1 span=c ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req5 @@ -257,25 +257,25 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req4 ---- num=2 lock: "b" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req5 @@ -375,12 +375,12 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 8, strength: Intent, txn: none active: true req: 9, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 8 @@ -394,11 +394,11 @@ release txn=txn1 span=b num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" - queued writers: + queued locking requests: active: false req: 9, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] @@ -412,15 +412,15 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" - queued writers: + queued locking requests: active: false req: 9, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 8, strength: Intent, txn: none distinguished req: 8 @@ -430,14 +430,14 @@ release txn=txn1 span=a ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 9, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 8, strength: Intent, txn: none distinguished req: 8 @@ -449,15 +449,15 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: false req: 9, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 8, strength: Intent, txn: none distinguished req: 8 @@ -469,13 +469,13 @@ acquire r=req7 k=b durability=u strength=exclusive ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 8, strength: Intent, txn: none distinguished req: 8 @@ -484,7 +484,7 @@ release txn=txn1 span=c ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] @@ -497,7 +497,7 @@ print ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] @@ -586,12 +586,12 @@ 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 writers: + queued locking requests: active: true req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 12 @@ -602,11 +602,11 @@ release txn=txn1 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)] - queued writers: + queued locking requests: active: true req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 lock: "b" - queued writers: + queued locking requests: active: false req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 # req11 reserves "a" @@ -614,10 +614,10 @@ release txn=txn1 span=a ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 # req11 breaks the reservation at "b" @@ -629,10 +629,10 @@ print ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - queued writers: + queued locking requests: active: false req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: false req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 @@ -644,11 +644,11 @@ acquire r=req11 k=b durability=u strength=exclusive ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 11, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: false req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 # req12 ignores the lock at "b" when it encounters it again as a reader. So it will @@ -671,7 +671,7 @@ dequeue r=req11 num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 12, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 12 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes index 5860b1985926..97a8ea8c3da8 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes @@ -67,7 +67,7 @@ acquire r=req2 k=a durability=r strength=intent num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 8.000000000,0, info: repl, unrepl [(str: Exclusive seq: 2)] - queued writers: + queued locking requests: active: true req: 1, strength: Intent, txn: none distinguished req: 1 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped index c53b90cf4599..3b7ad16c1282 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped @@ -96,7 +96,7 @@ num=1 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 2)] waiting readers: req: 1, txn: none - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: none distinguished req: 1 @@ -107,7 +107,7 @@ num=1 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl, unrepl [(str: Exclusive seq: 2)] waiting readers: req: 1, txn: none - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: none distinguished req: 1 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter index 538a82f4d0d0..971e61ac5f84 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter @@ -21,7 +21,7 @@ add-discovered r=req1 k=a txn=txn2 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 add-discovered r=req1 k=b txn=txn2 @@ -29,7 +29,7 @@ add-discovered r=req1 k=b txn=txn2 num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl @@ -39,13 +39,13 @@ add-discovered r=req1 k=c txn=txn2 num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 # req1 is not in the queue for "b" as readers are never inactive waiters. @@ -55,13 +55,13 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 new-request r=req2 txn=txn1 ts=10 spans=intent@c @@ -78,13 +78,13 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 @@ -102,14 +102,14 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: true req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 @@ -119,13 +119,13 @@ release txn=txn2 span=a ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 @@ -138,13 +138,13 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: true req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 @@ -155,12 +155,12 @@ release txn=txn2 span=c ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl lock: "c" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 @@ -176,7 +176,7 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl @@ -184,7 +184,7 @@ num=3 req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "c" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 @@ -194,10 +194,10 @@ release txn=txn2 span=b ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 1, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 @@ -209,7 +209,7 @@ dequeue r=req1 ---- num=1 lock: "c" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 guard-state r=req2 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write index ab8c13982e38..aeea9ea877fb 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write @@ -90,18 +90,18 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, strength: Intent, txn: none distinguished req: 2 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 @@ -112,15 +112,15 @@ release txn=txn1 span=a,d ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, strength: Intent, txn: none distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req2 @@ -152,16 +152,16 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, strength: Intent, txn: none active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 4 lock: "b" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 # Release the reservation at a. The first waiter is non-transactional so it will not acquire the @@ -172,13 +172,13 @@ dequeue r=req2 ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req4 @@ -193,15 +193,15 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, strength: Intent, txn: none distinguished req: 4 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 # Release the reservation at b. The non-transactional waiter will be done at b, and when it gets @@ -211,10 +211,10 @@ dequeue r=req3 ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req4 @@ -229,10 +229,10 @@ print ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 # Non-transactional request scans again and proceeds to evaluation and discovers a lock at c @@ -245,11 +245,11 @@ add-discovered r=req4 k=c txn=txn2 ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 4, strength: Intent, txn: none active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 @@ -275,10 +275,10 @@ release txn=txn2 span=c ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req4 @@ -295,17 +295,17 @@ dequeue r=req4 ---- num=2 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - queued writers: + queued locking requests: active: false req: 5, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req5 ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req6 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/query b/pkg/kv/kvserver/concurrency/testdata/lock_table/query index e89ebf63c0d1..a3dcd48e2d29 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/query +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/query @@ -136,14 +136,14 @@ print num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,2, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 4 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded b/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded index 92f0e789204e..c35378d6eaec 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded @@ -42,7 +42,7 @@ print 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 @@ -71,7 +71,7 @@ dequeue r=req4 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 @@ -99,7 +99,7 @@ dequeue r=req5 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 @@ -127,7 +127,7 @@ dequeue r=req6 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 @@ -152,7 +152,7 @@ dequeue r=req7 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 @@ -173,7 +173,7 @@ dequeue r=req8 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 @@ -200,7 +200,7 @@ acquire r=req9 k=b 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 @@ -226,13 +226,13 @@ 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 writers: + queued locking requests: active: true req: 2, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000006 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 10, strength: Intent, txn: 00000000-0000-0000-0000-000000000007 distinguished req: 10 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks b/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks index 468d491afca4..7190312c342c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/shared_locks @@ -62,7 +62,7 @@ print num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: false req: 3, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 active: false req: 4, strength: Shared, txn: 00000000-0000-0000-0000-000000000002 @@ -103,7 +103,7 @@ num=1 lock: "a" holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 5 @@ -127,7 +127,7 @@ num=1 lock: "a" holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 @@ -149,7 +149,7 @@ num=1 lock: "a" holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 5, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, strength: Shared, txn: 00000000-0000-0000-0000-000000000003 @@ -228,7 +228,7 @@ num=1 holders: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 12, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 distinguished req: 12 @@ -240,7 +240,7 @@ num=1 lock: "a" holders: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 12, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 distinguished req: 12 @@ -249,7 +249,7 @@ release txn=txn2 span=a num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 12, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 distinguished req: 12 @@ -259,7 +259,7 @@ release txn=txn3 span=a ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 12, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000004 @@ -293,4 +293,3 @@ num=1 # 8. Cases where the waiter is a non-transactional writer. For example, a state # transition that uncorks a number of non-transactional writer from the head of # the queue. - diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded index a48549a268b2..04c0c71df7d2 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded @@ -69,7 +69,7 @@ num=3 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl, unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 2, strength: Intent, txn: none distinguished req: 2 @@ -112,7 +112,7 @@ print num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 @@ -125,7 +125,7 @@ release txn=txn1 span=a ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "b" @@ -145,12 +145,12 @@ print ---- num=3 lock: "a" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 lock: "c" @@ -192,25 +192,25 @@ add-discovered r=req7 k=d txn=txn1 ---- num=4 lock: "a" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] waiting readers: req: 5, txn: 00000000-0000-0000-0000-000000000002 - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl, unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 7, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 8, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 new-request r=req8 txn=txn2 ts=10 spans=exclusive@e @@ -227,7 +227,7 @@ acquire r=req8 k=e durability=u strength=exclusive num=1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: repl - queued writers: + queued locking requests: active: false req: 8, strength: Intent, txn: 00000000-0000-0000-0000-000000000002 guard-state r=req2 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked index eb32c6748d93..0d7231ec6108 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked @@ -117,7 +117,7 @@ num=4 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "f" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 new-request r=req4 txn=txn1 ts=10,1 spans=shared@g @@ -141,7 +141,7 @@ num=5 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "f" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Shared seq: 0)] @@ -167,7 +167,7 @@ num=6 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "f" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Shared seq: 0)] @@ -195,7 +195,7 @@ num=7 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "f" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Shared seq: 0)] @@ -225,7 +225,7 @@ num=7 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "f" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Shared seq: 0)] @@ -233,7 +233,7 @@ num=7 holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 9.000000000,1, info: unrepl [(str: Shared seq: 0)] lock: "i" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 9.000000000,1, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 7 @@ -372,7 +372,7 @@ num=7 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "f" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Shared seq: 0)] @@ -380,7 +380,7 @@ num=7 holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 9.000000000,1, info: unrepl [(str: Shared seq: 0)] lock: "i" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 9.000000000,1, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 7 @@ -446,7 +446,7 @@ num=7 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Exclusive seq: 0)] lock: "f" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000001 lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 10.000000000,1, info: unrepl [(str: Shared seq: 0)] @@ -454,7 +454,7 @@ num=7 holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 9.000000000,1, info: unrepl [(str: Shared seq: 0)] lock: "i" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 9.000000000,1, info: unrepl [(str: Shared seq: 0)] - queued writers: + queued locking requests: active: true req: 7, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 7 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/update b/pkg/kv/kvserver/concurrency/testdata/lock_table/update index 458a49386cf4..c7ef221e8903 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/update +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/update @@ -97,7 +97,7 @@ num=1 waiting readers: req: 4, txn: none req: 2, txn: 00000000-0000-0000-0000-000000000002 - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 2 @@ -204,7 +204,7 @@ num=1 waiting readers: req: 4, txn: none req: 2, txn: 00000000-0000-0000-0000-000000000002 - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 2 @@ -222,7 +222,7 @@ num=1 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 13.000000000,1, info: unrepl [(str: Exclusive seq: 0)] waiting readers: req: 4, txn: none - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 4 @@ -242,7 +242,7 @@ num=1 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 13.000000000,1, info: unrepl [(str: Exclusive seq: 0)] waiting readers: req: 4, txn: none - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 4 @@ -259,7 +259,7 @@ num=1 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 13.000000000,1, info: unrepl [(str: Exclusive seq: 0)] waiting readers: req: 4, txn: none - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 4 @@ -275,7 +275,7 @@ num=1 holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 15.000000000,1, info: unrepl [(str: Exclusive seq: 0)] waiting readers: req: 4, txn: none - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 4 @@ -291,7 +291,7 @@ update txn=txn1 ts=17,1 epoch=0 span=a num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 17.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 3 @@ -309,7 +309,7 @@ dequeue r=req4 num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 17.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 3 @@ -323,7 +323,7 @@ update txn=txn1 ts=19,1 epoch=0 span=a num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, ts: 19.000000000,1, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 3 @@ -339,7 +339,7 @@ update txn=txn1 ts=19,1 epoch=1 span=a ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, strength: Intent, txn: none distinguished req: 5 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self b/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self index 6ebcd4c49e3c..49c9fa6780ee 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self @@ -74,7 +74,7 @@ print 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)] - queued writers: + queued locking requests: active: true req: 2, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 @@ -84,7 +84,7 @@ release txn=txn1 span=a ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 @@ -106,7 +106,7 @@ print ---- num=1 lock: "a" - queued writers: + queued locking requests: active: false req: 2, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000002 @@ -131,7 +131,7 @@ acquire r=req2 k=a durability=u strength=exclusive num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002 epoch: 0, iso: Serializable, ts: 10.000000000,0, info: unrepl [(str: Exclusive seq: 0)] - queued writers: + queued locking requests: active: true req: 3, strength: Intent, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 From 3c6df301382d8962cbd7affe11441df58b9dd1a3 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Thu, 24 Aug 2023 16:04:52 -0400 Subject: [PATCH 2/2] concurrency: s/l.mu/kl.mu/g We renamed the receiver when lockState was changed to keyLocks, but we never renamed all references to l.mu. Epic: none Release note: None --- pkg/kv/kvserver/concurrency/lock_table.go | 80 ++++++++++++----------- 1 file changed, 42 insertions(+), 38 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index c48ef0f51f8d..7186be875646 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -1234,7 +1234,7 @@ func newTxnLock(txn *enginepb.TxnMeta, clock *hlc.Clock) *txnLock { // getLockHolder returns information about the lock's holder. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (tl *txnLock) getLockHolder() (*enginepb.TxnMeta, hlc.Timestamp) { assert( tl.isHeldReplicated() || tl.isHeldUnreplicated(), @@ -1258,7 +1258,7 @@ func (tl *txnLock) getLockHolder() (*enginepb.TxnMeta, hlc.Timestamp) { // isHeldReplicated returns true if the receiver is held as a replicated lock. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (tl *txnLock) isHeldReplicated() bool { return !tl.replicatedInfo.ts.IsEmpty() } @@ -1266,7 +1266,7 @@ func (tl *txnLock) isHeldReplicated() bool { // isHeldUnreplicated returns true if the receiver is held as a unreplicated // lock. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (tl *txnLock) isHeldUnreplicated() bool { return !tl.unreplicatedInfo.ts.IsEmpty() } @@ -1275,7 +1275,7 @@ func (tl *txnLock) isHeldUnreplicated() bool { // by a transaction with multiple locking strengths, the mode corresponding to // the highest lock strength is returned. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (tl *txnLock) getLockMode() lock.Mode { lockHolderTxn, lockHolderTS := tl.getLockHolder() @@ -1302,7 +1302,7 @@ func (tl *txnLock) getLockMode() lock.Mode { // idempotent. Idempotent lock acquisitions do not require any changes to what // is being tracked in the lock's state. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (tl *txnLock) isIdempotentLockAcquisition(acq *roachpb.LockAcquisition) bool { txn, _ := tl.getLockHolder() assert(txn.ID == acq.Txn.ID, "existing lock transaction is different from the acquisition") @@ -1334,7 +1334,7 @@ func (tl *txnLock) isIdempotentLockAcquisition(acq *roachpb.LockAcquisition) boo // a lock is already held on the receiver's key by the transaction referenced in // the supplied lock acquisition. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (tl *txnLock) reacquireLock(acq *roachpb.LockAcquisition) error { // An unreplicated lock is being re-acquired... if acq.Durability == lock.Unreplicated && tl.isHeldUnreplicated() { @@ -1506,10 +1506,9 @@ type lockWaitQueue struct { // guaranteed to be isolated. We don't concern ourselves with the possible // fairness issue if the higher sequence number wins the race. // - // Non-locking readers are held in a separate list to the list of - // waitingReaders, and they make no claims on unheld locks like locking - // requests do. They race with the transactional locking requests that have - // made the claim. + // Non-locking readers are held separately, in the waitingReaders list. Unlike + // locking requests, they make no claims on unheld locks. Instead, they race + // with other locking request(s) that have made a claim. // // Similarly, non-transactional requests make no claims either, regardless of // their read/write status. Non-transactional writers wait in the @@ -1598,7 +1597,7 @@ func (kl *keyLocks) SetID(v uint64) { kl.id = v } func (kl *keyLocks) SetKey(v []byte) { kl.key = v } func (kl *keyLocks) SetEndKey(v []byte) { kl.endKey = v } -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) String() string { var sb redact.StringBuilder kl.safeFormat(&sb, nil) @@ -1606,7 +1605,7 @@ func (kl *keyLocks) String() string { } // SafeFormat implements redact.SafeFormatter. -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) SafeFormat(w redact.SafePrinter, _ rune) { var sb redact.StringBuilder kl.safeFormat(&sb, nil) @@ -1614,7 +1613,7 @@ func (kl *keyLocks) SafeFormat(w redact.SafePrinter, _ rune) { } // safeFormat is a helper for SafeFormat and String methods. -// REQUIRES: l.mu is locked. txnStatusCache can be nil. +// REQUIRES: kl.mu is locked. txnStatusCache can be nil. func (kl *keyLocks) safeFormat(sb *redact.StringBuilder, txnStatusCache *txnStatusCache) { sb.Printf(" lock: %s\n", kl.key) if kl.isEmptyLock() { @@ -1736,7 +1735,7 @@ func (kl *keyLocks) collectLockStateInfo( } // lockStateInfo converts receiver to the roachpb.LockStateInfo structure. -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) lockStateInfo(now time.Time) roachpb.LockStateInfo { var txnHolder *enginepb.TxnMeta @@ -1816,7 +1815,7 @@ func (kl *keyLocks) addToMetrics(m *LockTableMetrics, now time.Time) { // inconsistencies with waitSelf and waitForDistinguished states that need // changing. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) informActiveWaiters() { if kl.waitingReaders.Len() == 0 && kl.queuedLockingRequests.Len() == 0 { return // no active waiters to speak of; early return @@ -1937,7 +1936,7 @@ func (kl *keyLocks) informActiveWaiters() { // they use the concept of the transaction that has claimed a particular key as // the transaction to push. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) claimantTxn() (_ *enginepb.TxnMeta, held bool) { if kl.isLocked() { // We want the claimant transaction to remain the same unless there has been @@ -1980,7 +1979,7 @@ func (kl *keyLocks) releaseLockingRequestsFromTxn(txn *enginepb.TxnMeta) { // the waiting state for other actively waiting requests as well; as such, // informActiveWaiters is more appropriate. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) tryMakeNewDistinguished() { var g *lockTableGuardImpl claimantTxn, _ := kl.claimantTxn() @@ -2013,7 +2012,7 @@ func (kl *keyLocks) tryMakeNewDistinguished() { // Returns true iff the keyLocks is empty, i.e., there is no lock holder and no // waiters. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) isEmptyLock() bool { if kl.isLocked() { return false // lock is held @@ -2033,7 +2032,8 @@ func (kl *keyLocks) isEmptyLock() bool { // for a lock to be safely removed from the tree. If it does not hold, requests // with a stale snapshot of the btree will still be able to enter the lock's // wait-queue, after which point they will never hear of lock updates. -// REQUIRES: l.mu is locked. +// +// REQUIRES: kl.mu is locked. func (kl *keyLocks) assertEmptyLock() { if !kl.isEmptyLock() { panic("keyLocks is not empty") @@ -2041,7 +2041,8 @@ func (kl *keyLocks) assertEmptyLock() { } // assertEmptyLockUnlocked is like assertEmptyLock, but it locks the keyLocks. -// REQUIRES: l.mu is not locked. +// +// REQUIRES: kl.mu is not locked. func (kl *keyLocks) assertEmptyLockUnlocked() { kl.mu.Lock() defer kl.mu.Unlock() @@ -2052,7 +2053,7 @@ func (kl *keyLocks) assertEmptyLockUnlocked() { // held with durability replicated. If so, the first transaction that holds a // replicated lock is also returned. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) isAnyLockHeldReplicated() (bool, *enginepb.TxnMeta) { if !kl.isLocked() { return false, nil @@ -2067,7 +2068,8 @@ func (kl *keyLocks) isAnyLockHeldReplicated() (bool, *enginepb.TxnMeta) { } // Returns the duration of time the lock has been tracked as held in the lock table. -// REQUIRES: l.mu is locked. +// +// REQUIRES: kl.mu is locked. func (kl *keyLocks) lockHeldDuration(now time.Time) time.Duration { if !kl.isLocked() { return time.Duration(0) @@ -2085,7 +2087,7 @@ func (kl *keyLocks) lockHeldDuration(now time.Time) time.Duration { // Returns the total amount of time all waiters in the queues of // readers and locking requests have been waiting on the key referenced in the -// reciever. +// receiver. // // REQUIRES: kl.mu is locked. func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time.Duration) { @@ -2117,7 +2119,8 @@ func (kl *keyLocks) totalAndMaxWaitDuration(now time.Time) (time.Duration, time. // Returns true iff the lock is currently held by the transaction with the // given id. -// REQUIRES: l.mu is locked. +// +// REQUIRES: kl.mu is locked. func (kl *keyLocks) isLockedBy(id uuid.UUID) bool { if kl.isLocked() { _, ok := kl.heldBy[id] @@ -2130,7 +2133,7 @@ func (kl *keyLocks) isLockedBy(id uuid.UUID) bool { // or more transactions. Each transaction's locks may be held durably, // non-durably, or both. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) isLocked() bool { return kl.holders.Len() != 0 } @@ -2138,7 +2141,7 @@ func (kl *keyLocks) isLocked() bool { // clearLockHeldBy removes the lock, if held, by the transaction referenced by // the supplied ID. It is a no-op if the lock isn't held by the transaction. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) clearLockHeldBy(ID uuid.UUID) { e, held := kl.heldBy[ID] if !held { @@ -2157,7 +2160,7 @@ func (kl *keyLocks) clearAllLockHolders() { // acquired or discovered on the key referenced in the receiver for the first // time. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) lockAcquiredOrDiscovered(tl *txnLock) { _, found := kl.heldBy[tl.txn.ID] assert(!found, "lock was already being tracked for this key") @@ -2186,7 +2189,7 @@ func (kl *keyLocks) lockAcquiredOrDiscovered(tl *txnLock) { // evaluate. The guard's state is modified to indicate if there are locks that // need resolution. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) scanAndMaybeEnqueue(g *lockTableGuardImpl, notify bool) (wait bool) { kl.mu.Lock() defer kl.mu.Unlock() @@ -2256,7 +2259,7 @@ func (kl *keyLocks) scanAndMaybeEnqueue(g *lockTableGuardImpl, notify bool) (wai // constructWaitingState constructs the waiting state the supplied request // should use to wait in the receiver's lock wait-queues. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) constructWaitingState(g *lockTableGuardImpl) waitingState { waitForState := waitingState{ kind: waitFor, @@ -2281,7 +2284,7 @@ func (kl *keyLocks) constructWaitingState(g *lockTableGuardImpl) waitingState { // transaction already holds the lock with an equal or higher lock strength // compared to the one supplied. Otherwise, false is returned. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) alreadyHoldsLockAndIsAllowedToProceed( g *lockTableGuardImpl, str lock.Strength, ) bool { @@ -2325,7 +2328,7 @@ func (kl *keyLocks) alreadyHoldsLockAndIsAllowedToProceed( // non-conflicting. However, the caller may be responsible for cleaning them up // before proceeding. // -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. // REQUIRES: the transaction, to which the request belongs, should not be a lock // holder. func (kl *keyLocks) conflictsWithLockHolders(g *lockTableGuardImpl) bool { @@ -2441,7 +2444,7 @@ func (kl *keyLocks) conflictsWithLockHolders(g *lockTableGuardImpl) bool { // A boolean is returned indicating whether the read request conflicted with // the lock or not. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) maybeEnqueueNonLockingReadRequest(g *lockTableGuardImpl) (conflicts bool) { assert(g.curStrength() == lock.None, "unexpected locking strength; expected read") if !kl.conflictsWithLockHolders(g) { @@ -2468,7 +2471,7 @@ func (kl *keyLocks) maybeEnqueueNonLockingReadRequest(g *lockTableGuardImpl) (co // configured bound the request is not enqueued; instead, a boolean indicating // this case is returned to the caller. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) enqueueLockingRequest(g *lockTableGuardImpl) (maxQueueLengthExceeded bool) { assert(g.curStrength() != lock.None, "should only be called with a locking request") g.mu.Lock() @@ -2537,7 +2540,7 @@ func (kl *keyLocks) enqueueLockingRequest(g *lockTableGuardImpl) (maxQueueLength // [1] A request that belongs to the lock's claimant transaction is not eligible // to become a distinguished waiter. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) maybeMakeDistinguishedWaiter(g *lockTableGuardImpl) { if kl.distinguishedWaiter != nil { return @@ -2553,7 +2556,7 @@ func (kl *keyLocks) maybeMakeDistinguishedWaiter(g *lockTableGuardImpl) { // shouldRequestActivelyWait returns true iff the supplied request needs to // actively wait on the receiver. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. // REQUIRES: g.mu to be locked. func (kl *keyLocks) shouldRequestActivelyWait(g *lockTableGuardImpl) bool { if g.curStrength() == lock.None { @@ -2609,7 +2612,7 @@ func (kl *keyLocks) shouldRequestActivelyWait(g *lockTableGuardImpl) bool { // [2] While non-transactional writers cannot establish claims, they do need to // be removed from the receiver's wait queue before proceeding. We do that here. // -// REQUIRES: l.mu to be locked. +// REQUIRES: kl.mu to be locked. func (kl *keyLocks) claimBeforeProceeding(g *lockTableGuardImpl) { assert(g.curStrength() != lock.None, "non-locking requests should not try to grab claims") @@ -2989,7 +2992,7 @@ func (kl *keyLocks) tryUpdateLock(up *roachpb.LockUpdate) (heldByTxn, gc bool) { return kl.tryUpdateLockLocked(*up) } -// REQUIRES: l.mu is locked. +// REQUIRES: kl.mu is locked. func (kl *keyLocks) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bool) { if kl.isEmptyLock() { // Already free. This can happen when an unreplicated lock is removed in @@ -3108,7 +3111,8 @@ func (kl *keyLocks) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bo // The lock holder timestamp has increased. Some of the waiters may no longer // need to wait. -// REQUIRES: l.mu is locked. +// +// REQUIRES: kl.mu is locked. func (kl *keyLocks) increasedLockTs(newTs hlc.Timestamp) { distinguishedRemoved := false for e := kl.waitingReaders.Front(); e != nil; {