From ff9c3573dfd3ea84ba6dd91a802388a7e754a8c6 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Mon, 15 May 2023 23:34:56 -0400 Subject: [PATCH] concurrency: get rid of reservations in the lock table This patch removes the notion of reservations from the lock table. Reservations served as a claim that prevented multiple requests from racing when a lock was released. Typically, when a lock was released, only the first transactional writer was released from the list of queued writers. It would do so by claiming a "reservation" on the lock. All requests that are sequenced through the lock table are associated with a sequence number based on arrival order. These sequence numbers are used to uphold ~fairness as requests are sequenced. They also serve a correctness purpose -- because all locks are not known upfront (as uncontended replicated locks may be discovered during evaluation), sequence numbers are used to break potential deadlocks that arise from out of order locking. This motivated the concept of reservation breaking, which could happen if a lower sequence number request encountered a reservation by a request with a higher sequence number. This would lead to somewhat complex state management, where requests could move from being reservations to inactive waiters multiple times during their lifetime. A lot of this can be simplified if we make no distinction between a reservation and an inactive waiter. This patch gets rid of reservations entirely. Instead, it offers a new invariant: The head of the list of waiting writers should always be an inactive, transactional writer if the lock isn't held. In practice, this works out functionally the same as how reservations operated, albeit with fewer state transitions. Being an inactive waiter at the head of the lock's wait-queue serves as the request's claim on the key. As such, verbiage that referenced "reservations" previously is now updated to talk about claims and claimant transactions. There's a bit of comment churn as a result. There's also some datadriven test churn as part of this patch -- but it should be helpful in convincing ourselves that this just changes concepts, and not functionality. In particular, what was previously a reservation holder, is now the first inactive queued writer at the lock. Closes #103361 Release note: None --- .../concurrency/concurrency_control.go | 25 +- .../concurrency/concurrency_manager.go | 30 +- pkg/kv/kvserver/concurrency/lock_table.go | 690 ++++++++---------- .../kvserver/concurrency/lock_table_waiter.go | 68 +- .../concurrency/lock_table_waiter_test.go | 9 +- pkg/kv/kvserver/concurrency/metrics.go | 13 +- .../clear_abandoned_intents | 27 +- .../testdata/concurrency_manager/deadlocks | 29 +- .../testdata/concurrency_manager/lock_timeout | 8 +- .../testdata/concurrency_manager/priority | 6 +- .../concurrency_manager/queue_length_exceeded | 6 +- .../concurrency_manager/range_state_listener | 15 +- .../concurrency_manager/wait_policy_error | 8 +- .../concurrency_manager/wait_policy_skip | 3 +- .../testdata/concurrency_manager/wait_self | 6 +- .../testdata/lock_table/add_discovered | 5 +- .../concurrency/testdata/lock_table/basic | 535 ++++++++------ .../lock_table/clear_finalized_txn_locks | 40 +- .../concurrency/testdata/lock_table/disable | 6 +- .../discovered_locks_consults_txn_cache | 12 +- .../testdata/lock_table/dup_access | 85 ++- .../testdata/lock_table/non_active_waiter | 24 +- .../testdata/lock_table/non_txn_write | 63 +- .../testdata/lock_table/size_limit_exceeded | 6 +- .../testdata/lock_table/skip_locked | 9 +- .../concurrency/testdata/lock_table/update | 2 +- .../concurrency/testdata/lock_table/wait_self | 4 +- pkg/storage/mvcc.go | 24 +- 28 files changed, 937 insertions(+), 821 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index 2a453afa12b8..93e64524d385 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -772,15 +772,22 @@ type lockTableGuard interface { // that conflict. CheckOptimisticNoConflicts(*lockspanset.LockSpanSet) (ok bool) - // IsKeyLockedByConflictingTxn returns whether the specified key is locked or - // reserved (see lockTable "reservations") by a conflicting transaction in the - // lockTableGuard's snapshot of the lock table, given the caller's own desired - // locking strength. If so, true is returned. If the key is locked, the lock - // holder is also returned. Otherwise, if the key is reserved, nil is also - // returned. A transaction's own lock or reservation does not appear to be - // locked to itself (false is returned). The method is used by requests in - // conjunction with the SkipLocked wait policy to determine which keys they - // should skip over during evaluation. + // IsKeyLockedByConflictingTxn returns whether the specified key is claimed + // (see claimantTxn()) by a conflicting transaction in the lockTableGuard's + // snapshot of the lock table, given the caller's own desired locking + // strength. If so, true is returned. If the key is locked, the lock holder is + // also returned. Otherwise, if the key was claimed by a concurrent request + // still sequencing through the lock table, but the lock isn't held (yet), nil + // is also returned. + // + // If the lock has been claimed (held or otherwise) by the transaction itself, + // there's no conflict to speak of, so false is returned. In cases where the + // lock isn't held, but the lock has been claimed by the transaction itself, + // we do not make a distinction about which request claimed the key -- it + // could either be the request itself, or a different concurrent request from + // the same transaction; The specifics do not affect the caller. + // This method is used by requests in conjunction with the SkipLocked wait + // policy to determine which keys they should skip over during evaluation. IsKeyLockedByConflictingTxn(roachpb.Key, lock.Strength) (bool, *enginepb.TxnMeta) } diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index dd356c5b3bc7..a3bec9a2893c 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -412,9 +412,10 @@ func (m *managerImpl) PoisonReq(g *Guard) { func (m *managerImpl) FinishReq(g *Guard) { // NOTE: we release latches _before_ exiting lock wait-queues deliberately. // Either order would be correct, but the order here avoids non-determinism in - // cases where a request A holds both latches and lock wait-queue reservations - // and has a request B waiting on its reservations. If request A released its - // reservations before releasing its latches, it would be possible for B to + // cases where a request A holds both latches and has claimed some keys by + // virtue of being the first request in a lock wait-queue and has a request B + // waiting on its claim. If request A released its claim (by exiting the lock + // wait-queue) before releasing its latches, it would be possible for B to // beat A to the latch manager and end up blocking on its latches briefly. Not // only is this confusing in traces, but it is slightly less efficient than if // request A released latches before letting anyone waiting on it in the lock @@ -759,15 +760,22 @@ func (g *Guard) CheckOptimisticNoLatchConflicts() (ok bool) { return g.lm.CheckOptimisticNoConflicts(g.lg, g.Req.LatchSpans) } -// IsKeyLockedByConflictingTxn returns whether the specified key is locked or -// reserved (see lockTable "reservations") by a conflicting transaction in the -// Guard's snapshot of the lock table, given the caller's own desired locking +// IsKeyLockedByConflictingTxn returns whether the specified key is claimed +// (see claimantTxn()) by a conflicting transaction in the lockTableGuard's +// snapshot of the lock table, given the caller's own desired locking // strength. If so, true is returned. If the key is locked, the lock holder is -// also returned. Otherwise, if the key is reserved, nil is also returned. A -// transaction's own lock or reservation does not appear to be locked to itself -// (false is returned). The method is used by requests in conjunction with the -// SkipLocked wait policy to determine which keys they should skip over during -// evaluation. +// also returned. Otherwise, if the key was claimed by a concurrent request +// still sequencing through the lock table, but the lock isn't held (yet), nil +// is also returned. +// +// If the lock has been claimed (held or otherwise) by the transaction itself, +// there's no conflict to speak of, so false is returned. In cases where the +// lock isn't held, but the lock has been claimed by the transaction itself, +// we do not make a distinction about which request claimed the key -- it +// could either be the request itself, or a different concurrent request from +// the same transaction; The specifics do not affect the caller. +// This method is used by requests in conjunction with the SkipLocked wait +// policy to determine which keys they should skip over during evaluation. func (g *Guard) IsKeyLockedByConflictingTxn( key roachpb.Key, strength lock.Strength, ) (bool, *enginepb.TxnMeta) { diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 42f7b8ef7c78..e00609e3e32f 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -60,10 +60,16 @@ const ( // However, sequencing information inside the lockTable is mostly discarded. waitElsewhere - // waitSelf indicates that a different requests from the same transaction - // has a conflicting reservation. See the comment about "Reservations" in - // lockState. This request should sit tight and wait for a new notification - // without pushing anyone. + // waitSelf indicates that a different request from the same transaction has + // claimed the lock already. This request should sit tight and wait for a new + // notification without pushing anyone. + // + // By definition, the lock cannot be held at this point -- if it were, another + // request from the same transaction would not be in the lock's wait-queues, + // obviating the need for this state. + // + // TODO(arul): this waitSelf state + claimantTxn stuff won't extend well to + // multiple lock holders. See TODO in informActiveWaiters. waitSelf // waitQueueMaxLengthExceeded indicates that the request attempted to enter a @@ -214,8 +220,8 @@ type lockTableImpl struct { // Example 2: // - Same as example 1 but lock at A is held by txn3 and lock at B is held // by txn4. - // - Lock at A is released so req1 acquires the reservation at A and starts - // waiting at B. + // - Lock at A is released so req1 claims the lock at A and starts waiting at + // B. // - It is unfair for req1 to wait behind req2 at B. The sequence number // assigned to req1 and req2 will restore the fairness by making req1 // wait before req2. @@ -228,11 +234,11 @@ type lockTableImpl struct { // It proceeds to evaluation and acquires the lock at A for txn2 and then // the request is done. The lock is still held. // - req3 (from txn3) wants to write to A and B. It queues at A. - // - txn2 releases A. req3 is in the front of the queue at A and gets the - // reservation and starts waiting at B behind req1. - // - txn0 releases B. req1 gets the reservation at B and does another scan - // and adds itself to the queue at A, behind req3 which holds the - // reservation at A. + // - txn2 releases A. req3 is in the front of the queue at A so it claims the + // lock and starts waiting at B behind req1. + // - txn0 releases B. req1 gets to claim the lock at B and does another scan + // and adds itself to the queue at A, behind req3 which holds the claim for + // A. // Now in the queues for A and B req1 is behind req3 and vice versa and // this deadlock has been created entirely due to the lock table's behavior. // TODO(nvanbenschoten): use an atomic.Uint64. @@ -348,8 +354,8 @@ func (t *lockTableImpl) setMaxLocks(maxLocks int64) { // lockTable is mostly discarded. // // - The waitSelf state is a rare state when a different request from the same -// transaction has a reservation. See the comment about "Reservations" in -// lockState. +// transaction has claimed the lock. See the comment about the concept of +// 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 @@ -374,14 +380,14 @@ type lockTableGuardImpl struct { // the lockStates in this snapshot may have been removed from // lockTableImpl. Additionally, it is possible that there is a new lockState // for the same key. This can result in various harmless anomalies: - // - the request may hold a reservation on a lockState that is no longer + // - the request may hold a claim on a lockState that is no longer // in the tree. When it next does a scan, it will either find a new // lockState where it will compete or none. Both lockStates can be in // the mu.locks map, which is harmless. - // - the request may wait behind a reservation holder that is not the - // lock holder. This could cause a delay in pushing the lock holder. - // This is not a correctness issue (the whole system is not deadlocked) - // and we expect will not be a real performance issue. + // - the request may wait behind a transaction that has claimed a lock but is + // yet to acquire it. This could cause a delay in pushing the lock holder. + // This is not a correctness issue (the whole system is not deadlocked) and we + // expect will not be a real performance issue. // // TODO(sbhola): experimentally evaluate the lazy queueing of the current // implementation, in comparison with eager queueing. If eager queueing @@ -410,9 +416,9 @@ type lockTableGuardImpl struct { // release etc.) may cause the request to no longer need to wait at this // key. It then needs to continue iterating through spans to find the next // key to wait at (we don't want to wastefully start at the beginning since - // this request probably has a reservation at the contended keys there): sa, - // ss, index, key collectively track the current position to allow it to - // continue iterating. + // this request probably has a claim at the contended keys there): str, index, + // and key collectively track the current position to allow it to continue + // iterating. // The key for the lockState. key roachpb.Key @@ -436,11 +442,7 @@ type lockTableGuardImpl 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. This map also includes locks for which the - // request holds a reservation. - // - // TODO(arul): this reservation comment will be stale when we remove the - // concept of reservations. + // 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 lockState @@ -460,8 +462,8 @@ type lockTableGuardImpl struct { // signaled, but what the state should be has not been computed. The call // to CurState() needs to compute that current state. Deferring the // computation makes the waiters do this work themselves instead of making - // the call to release/update locks or release reservations do this work - // (proportional to number of waiters). + // the call to release locks or update locks or remove the request's claims + // on (unheld) locks. This is proportional to number of waiters. mustFindNextLockAfter bool } // Locks to resolve before scanning again. Doesn't need to be protected by @@ -613,24 +615,21 @@ func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn( // The lock is empty but has not yet been deleted. return false, nil } - if !l.holder.locked { - // Key reserved. + conflictingTxn, held := l.claimantTxn() + assert(conflictingTxn != nil, "non-empty lockState with no claimant transaction") + if !held { if strength == lock.None { - // Non-locking reads only care about locks, not reservations. + // Non-locking reads only care about locks that are held. return false, nil } - if g.isSameTxn(l.reservation.txn) { - // Already reserved by this txn. + if g.isSameTxn(conflictingTxn) { return false, nil } - // "If the key is reserved, nil is returned." + // If the key is claimed but the lock isn't held (yet), nil is returned. return true, nil } // Key locked. txn, ts := l.getLockHolder() - if txn == nil { - panic("non-empty lockState with nil lock holder and nil reservation") - } if strength == lock.None && g.ts.Less(ts) { // Non-locking read below lock's timestamp. return false, nil @@ -665,6 +664,7 @@ func (g *lockTableGuardImpl) isSameTxn(txn *enginepb.TxnMeta) bool { return g.txn != nil && g.txn.ID == txn.ID } +// TODO(arul): get rid of this once tryActiveWait is cleaned up. func (g *lockTableGuardImpl) isSameTxnAsReservation(ws waitingState) bool { return !ws.held && g.isSameTxn(ws.txn) } @@ -755,16 +755,19 @@ func (g *lockTableGuardImpl) findNextLockAfter(notify bool) { // writer is typically waiting in an active state, i.e., the // lockTableGuardImpl.key refers to this lockState. However, there are // multiple reasons that can cause a writer to be an inactive waiter: -// - Breaking of reservations (see the comment on reservations below, in -// lockState) 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. +// - 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 +// 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). // - A lock held by a finalized txn causes the first waiter to be an inactive // waiter. // -// The first case above (breaking reservations) only occurs for transactional -// requests, but the other cases can happen for both transactional and -// non-transactional requests. +// 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. type queuedGuard struct { guard *lockTableGuardImpl active bool // protected by lockState.mu @@ -819,6 +822,10 @@ type lockState struct { // - !holder.locked => waitingReaders.Len() == 0. That is, readers wait // only if the lock is held. They do not wait for a reservation. // - If reservation != nil, that request is not in queuedWriters. + // + // TODO(arul): These invariants are stale now that we don't have reservations. + // However, we'll replace this structure soon with what's proposed in the + // SHARED locks RFC, at which point there will be new invariants altogether. // Information about whether the lock is held and the holder. We track // information for each durability level separately since a transaction can @@ -851,72 +858,92 @@ type lockState struct { } type lockWaitQueue struct { - // Reservations: + // TODO(sbhola): There are a number of places where we iterate over these + // lists looking for something, as described below. If some of these turn + // out to be inefficient, consider better data-structures. One idea is that + // 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. + // - 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. + // + // waitingReaders: + // - readers with a higher timestamp than some timestamp. + // - to find a particular guard. + + // Waiters: An active waiter needs to be notified about changes in who it is + // waiting for. + + // 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. + // + // 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: // - // A not-held lock can be "reserved". A reservation is just a claim that - // prevents multiple requests from racing when the lock is released. A - // reservation by req2 can be broken by req1 is req1 has a smaller seqNum - // than req2. Only requests that specify SpanReadWrite for a key can make - // reservations. This means a reservation can only be made when the lock is - // not held, since the reservation (which can acquire an Exclusive lock) and - // the lock holder (which is an Exclusive lock) conflict. + // 1. It helps ensure some degree of fairness, as sequence numbers are a proxy + // for arrival order. + // 2. Perhaps more importantly, enforcing this ordering helps prevent + // range-local lock table deadlocks. This is because all locks aren't known + // 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. // - // Read reservations are not permitted due to the complexities discussed in - // the review for #43740. Additionally, reads do not queue for their turn at - // all -- they are held in the waitingReaders list while the lock is held - // and removed when the lock is not released, so they race with - // reservations. Let us consider scenarios where reads did wait in the same - // queue: the lock could be held or reserved by a write at ts=20, followed - // by a waiting writer at ts=18, writer at ts=10, reader at ts=12. That - // reader is waiting not because of a conflict with the holder, or reserver, - // or the first waiter, but because there is a waiter ahead of it which it - // conflicts with. This introduces more complexity in tracking who this - // reader should push. Also consider a scenario where a reader did not wait - // in the queue and waited on the side like in waitingReaders but acquired a - // read reservation (together with other readers) when the lock was - // released. Ignoring the unfairness of this, we can construct a deadlock - // scenario with request req1 with seqnum 1 and req2 with seqnum 2 where - // req1 and req2 both want to write at one key and so get ordered by their - // seqnums but at another key req2 wants to read and req1 wants to write and - // since req2 does not wait in the queue it acquires a read reservation - // before req1. See the discussion at the end of this comment section on how - // the behavior will extend when we start supporting Shared and Update - // locks. + // 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. // - // Non-transactional requests can do both reads and writes but cannot be - // depended on since they don't have a transaction that can be pushed. - // Therefore they not only do not acquire locks, but cannot make reservations. - // The non-reservation for reads is already covered in the previous - // paragraph. For non-transactional writes, the request waits in the queue - // with other writers. The difference occurs: - // - when it gets to the front of the queue and there is no lock holder - // or reservation: instead of acquiring the reservation it removes - // itself from the lockState and proceeds to the next lock. If it - // does not need to wait for any more locks and manages to acquire - // latches before those locks are acquired by some other request, it - // will evaluate. - // - when deciding to wait at a lock: if the lock has a reservation with - // a sequence num higher than this non-transactional request it will - // ignore that reservation. Note that ignoring such reservations is - // safe since when this non-transactional request is holding latches - // those reservation holders cannot be holding latches, so they cannot - // conflict. + // 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. + // + // 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: + // 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. + // 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, + // it'll wait. // // Multiple requests from the same transaction wait independently, including - // the situation where one of the requests has a reservation and the other - // is waiting (currently this can only happen if both requests are doing - // SpanReadWrite). Making multiple requests from the same transaction - // jointly hold the reservation introduces code complexity since joint - // reservations can be partially broken (see deadlock example below), and is - // not necessarily fair to other requests. Additionally, if req1 from txn1 - // is holding a a reservation and req2 from txn1 is waiting, they must - // conflict wrt latches and cannot evaluate concurrently so there isn't a - // benefit to joint reservations. However, if one of the requests acquires - // the lock the other request no longer needs to wait on this lock. This - // situation motivates the waitSelf state. + // 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. + // + // 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". // - // Deadlock example if joint reservations were supported and we did not - // allow partial breaking of such reservations: + // Once we introduce joint claims, we'll also need to support partially + // breaking such claims. This means that a request that was previously + // marked as inactive may have to come back to a lock and actively wait on it. + // Here's a sketch of what a deadlock could look like if this wasn't + // supported: // // - Keys are A, B, C, D. // - Key D is locked by some random txn. @@ -928,94 +955,26 @@ type lockWaitQueue struct { // evaluates, and locks A. // - req3 from txn1 that writes A, C. It waits at A. Note that req1 and req3 // are from the same txn. - // - A is unlocked. req3 reserves A and waits at C behind req2. + // - A is unlocked. req3 claims A and waits at C behind req2. // - B is locked by some random txn. - // - D is unlocked. req1 reserves D and proceeds to scan again and finds A - // is reserved by req3 which is the same txn so becomes a joint - // reservation holder at A. + // - D is unlocked. req1 claims D and proceeds to scan again and finds A + // is claimed by req3 which is the same txn so becomes a joint + // claim holder at A. // - Since B is locked, req1 waits at B. - // - C is unlocked. req2 reserves C. It scans and finds req1+req3 holding - // the joint reservation at A. If it queues behind this joint reservation + // - C is unlocked. req2 claims C. It scans and finds req1+req3 holding + // the joint claim at A. If it queues behind this joint claim // we have the following situation: - // reservation waiter + // claim waiter // A req1+req3 req2 // C req2 req3 // This is a deadlock caused by the lock table unless req2 partially - // breaks the reservation at A. - // - // Extension for Shared and Update locks: - // There are 3 aspects to consider: holders; reservers; the dependencies - // that need to be captured when waiting. - // - // - Holders: only shared locks are compatible with themselves, so there can - // be one of (a) no holder (b) multiple shared lock holders, (c) one - // exclusive holder, (d) one upgrade holder. Non-locking reads will - // wait in waitingReaders for only an incompatible exclusive holder. - // - // - Reservers: This follows the same pattern as holders. Non-locking reads - // do not wait on reservers. - // - // - Queueing and dependencies: All potential lockers and non-transactional - // writers will wait in the same queue. A sequence of consecutive requests - // that have the potential to acquire a shared lock will jointly reserve - // that shared lock. Such requests cannot jump ahead of requests with a - // lower seqnum just because there is currently a shared lock reservation - // (this can cause lockTable induced deadlocks). Such joint reservations - // can be partially broken by a waiter desiring an exclusive or upgrade - // lock. Like the current code, non-transactional writes will wait for - // reservations that have a lower sequence num, but not make their own - // reservation. Additionally, they can partially break joint reservations. - // - // Reservations that are (partially or fully) broken cause requests to - // reenter the queue as inactive waiters. This is no different than the - // current behavior. Each request can specify the same key in spans for - // ReadOnly, ReadShared, ReadUpgrade, ReadWrite. The spans will be - // iterated over in decreasing order of strength, to only wait at a lock - // at the highest strength (this is similar to the current behavior using - // accessDecreasingStrength). - // - // For dependencies, a waiter desiring an exclusive or upgrade lock always - // conflicts with the holder(s) or reserver(s) so that is the dependency - // that will be captured. A waiter desiring a shared lock may encounter a - // situation where it does not conflict with the holder(s) or reserver(s) - // since those are also shared lockers. In that case it will depend on the - // first waiter since that waiter must be desiring a lock that is - // incompatible with a shared lock. - // - // TODO(arul): The paragraph above still talks about declaring access on keys - // in terms of SpanAccess instead of lock strength. Switch over this verbiage - // to reference locking/non-locking requests once we support multiple lock - // strengths and add support for joint reservations. - - reservation *lockTableGuardImpl - - // TODO(sbhola): There are a number of places where we iterate over these - // lists looking for something, as described below. If some of these turn - // out to be inefficient, consider better data-structures. One idea is that - // 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. - // - 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. - // - // waitingReaders: - // - readers with a higher timestamp than some timestamp. - // - to find a particular guard. - - // Waiters: An active waiter needs to be notified about changes in who it is - // waiting for. - - // List of *queuedGuard. A subset of these are actively waiting. If - // non-empty, either the lock is held or there is a reservation. + // breaks the claim at A. queuedWriters 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 - // waitSelf state since that state is only used when there is a reservation. + // waitSelf state since that requests don't conflict with locks held by their + // transaction. waitingReaders list.List // If there is a non-empty set of active waiters that are not waitSelf, then @@ -1057,12 +1016,6 @@ func (l *lockState) safeFormat(sb *redact.StringBuilder, finalizedTxnCache *txnC sb.SafeString(" empty\n") return } - writeResInfo := func(sb *redact.StringBuilder, txn *enginepb.TxnMeta, ts hlc.Timestamp) { - // TODO(sbhola): strip the leading 0 bytes from the UUID string since tests are assigning - // UUIDs using a counter and makes this output more readable. - sb.Printf("txn: %v, ts: %v, seq: %v\n", - redact.Safe(txn.ID), redact.Safe(ts), redact.Safe(txn.Sequence)) - } writeHolderInfo := func(sb *redact.StringBuilder, txn *enginepb.TxnMeta, ts hlc.Timestamp) { sb.Printf(" holder: txn: %v, ts: %v, info: ", redact.Safe(txn.ID), redact.Safe(ts)) first := true @@ -1102,10 +1055,7 @@ func (l *lockState) safeFormat(sb *redact.StringBuilder, finalizedTxnCache *txnC sb.SafeString("\n") } txn, ts := l.getLockHolder() - if txn == nil { - sb.Printf(" res: req: %d, ", l.reservation.seqNum) - writeResInfo(sb, l.reservation.txn, l.reservation.ts) - } else { + if txn != nil { writeHolderInfo(sb, txn, ts) } // TODO(sumeer): Add an optional `description string` field to Request and @@ -1151,14 +1101,22 @@ func (l *lockState) collectLockStateInfo( l.mu.Lock() defer l.mu.Unlock() - // Don't include locks that have neither lock holders, nor reservations, nor + // Don't include locks that have neither lock holders, nor claims, nor // waiting readers/writers. if l.isEmptyLock() { return false, roachpb.LockStateInfo{} } - // Filter out locks without waiting readers/writers unless explicitly requested. - if !includeUncontended && l.waitingReaders.Len() == 0 && l.queuedWriters.Len() == 0 { + // Filter out locks without waiting readers/writers 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. + // See https://github.com/cockroachdb/cockroach/issues/103894. + if !includeUncontended && l.waitingReaders.Len() == 0 && + (l.queuedWriters.Len() == 0 || + (l.queuedWriters.Len() == 1 && !l.queuedWriters.Front().Value.(*queuedGuard).active)) { return false, roachpb.LockStateInfo{} } @@ -1181,25 +1139,9 @@ func (l *lockState) lockStateInfo(now time.Time) roachpb.LockStateInfo { } waiterCount := l.waitingReaders.Len() + l.queuedWriters.Len() - hasReservation := l.reservation != nil && l.reservation.txn != nil - if hasReservation { - waiterCount++ - } lockWaiters := make([]lock.Waiter, 0, waiterCount) - // Consider the reservation as the "first waiter" (albeit on an unheld lock). - if hasReservation { - l.reservation.mu.Lock() - lockWaiters = append(lockWaiters, lock.Waiter{ - WaitingTxn: l.reservation.txn, - ActiveWaiter: false, - Strength: lock.Exclusive, - WaitDuration: now.Sub(l.reservation.mu.curLockWaitStart), - }) - l.reservation.mu.Unlock() - } - - // Next, add waiting readers before writers as they should run first. + // Add waiting readers before writers as they should run first. for e := l.waitingReaders.Front(); e != nil; e = e.Next() { readerGuard := e.Value.(*lockTableGuardImpl) readerGuard.mu.Lock() @@ -1256,33 +1198,28 @@ func (l *lockState) addToMetrics(m *LockTableMetrics, now time.Time) { m.addLockMetrics(lm) } -// Called for a write request when there is a reservation. Returns true iff it -// succeeds. -// REQUIRES: l.mu is locked. -func (l *lockState) tryBreakReservation(seqNum uint64) bool { - if l.reservation.seqNum > seqNum { - qg := &queuedGuard{ - guard: l.reservation, - active: false, - } - l.queuedWriters.PushFront(qg) - l.reservation = nil - return true - } - return false -} - -// Informs active waiters about reservation or lock holder. The reservation -// may have changed so this needs to fix any inconsistencies wrt waitSelf and -// waitForDistinguished states. +// informActiveWaiters informs active waiters about the transaction that has +// claimed the lock. The claimant transaction may have changed, so there may be +// inconsistencies with waitSelf and waitForDistinguished states that need +// changing. // REQUIRES: l.mu is locked. func (l *lockState) informActiveWaiters() { + if l.waitingReaders.Len() == 0 && l.queuedWriters.Len() == 0 { + return // no active waiters to speak of; early return + } waitForState := waitingState{ kind: waitFor, key: l.key, queuedWriters: l.queuedWriters.Len(), queuedReaders: l.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 + // queue, but because there is a waiter with a lower sequence number that it + // is incompatible with. In such cases, its this guy it should be pushing. + // However, if we naively plugged things into the current structure, it would + // either sit tight (because its waiting for itself) or, worse yet, push a + // transaction it's actually compatible with! waitForState.txn, waitForState.held = l.claimantTxn() findDistinguished := false // We need to find a (possibly new) distinguished waiter if either: @@ -1385,7 +1322,18 @@ func (l *lockState) claimantTxn() (_ *enginepb.TxnMeta, held bool) { if lockHolderTxn, _ := l.getLockHolder(); lockHolderTxn != nil { return lockHolderTxn, true } - return l.reservation.txn, false + if l.queuedWriters.Len() == 0 { + panic("no queued writers or lock holder; no one should be waiting on the lock") + } + qg := l.queuedWriters.Front().Value.(*queuedGuard) + if qg.active || qg.guard.txn == nil { + // TODO(arul): uncomment this assertion once tryActiveWait has been + // refactored, and we no longer call into this method before readjusting + // the queued of writers to make the first one inactive. + //panic("first queued writer should be transactional and inactive") + return qg.guard.txn, false + } + return qg.guard.txn, false } // releaseWritersFromTxn removes all waiting writers for the lockState that are @@ -1406,15 +1354,25 @@ func (l *lockState) releaseWritersFromTxn(txn *enginepb.TxnMeta) { // When the active waiters have shrunk and the distinguished waiter has gone, // try to make a new distinguished waiter if there is at least 1 active // waiter. +// +// This function should only be called if the claimant transaction has +// not changed. This is asserted below. If the claimant transaction has changed, +// we not only need to find a new distinguished waiter, we also need to update +// the waiting state for other actively waiting requests as well; as such, +// informActiveWaiters is more appropriate. +// // REQUIRES: l.mu is locked. func (l *lockState) tryMakeNewDistinguished() { var g *lockTableGuardImpl + claimantTxn, _ := l.claimantTxn() if l.waitingReaders.Len() > 0 { g = l.waitingReaders.Front().Value.(*lockTableGuardImpl) } else if l.queuedWriters.Len() > 0 { for e := l.queuedWriters.Front(); e != nil; e = e.Next() { qg := e.Value.(*queuedGuard) - if qg.active && (l.reservation == nil || !qg.guard.isSameTxn(l.reservation.txn)) { + // Only requests actively waiting at this lock should be considered for + // the distinguished distinction. + if qg.active && !qg.guard.isSameTxn(claimantTxn) { g = qg.guard break } @@ -1423,6 +1381,9 @@ func (l *lockState) tryMakeNewDistinguished() { if g != nil { l.distinguishedWaiter = g g.mu.Lock() + assert( + g.mu.state.txn.ID == claimantTxn.ID, "tryMakeNewDistinguished called with new claimant txn", + ) g.mu.state.kind = waitForDistinguished // The rest of g.state is already up-to-date. g.notify() @@ -1430,22 +1391,23 @@ func (l *lockState) tryMakeNewDistinguished() { } } -// Returns true iff the lockState is empty, i.e., there is no lock holder or -// reservation. +// Returns true iff the lockState is empty, i.e., there is no lock holder and no +// waiters. // REQUIRES: l.mu is locked. func (l *lockState) isEmptyLock() bool { - if !l.holder.locked && l.reservation == nil { - for i := range l.holder.holder { - if !l.holder.holder[i].isEmpty() { - panic("lockState with !locked but non-zero lockHolderInfo") - } - } - if l.waitingReaders.Len() > 0 || l.queuedWriters.Len() > 0 { - panic("lockState with waiters but no holder or reservation") - } - return true + if l.holder.locked { + return false // lock is held } - return false + // The lock isn't held. Sanity check the lock state is sane: + // 1. Lock holder information should be zero-ed out. + // 2. There should be no waiting readers. + for i := range l.holder.holder { + assert(l.holder.holder[i].isEmpty(), "lockState with !locked but non-zero lockHolderInfo") + } + assert(l.waitingReaders.Len() == 0, "lockState with waiting readers but no holder") + // Determine if the lock is empty or not by checking the list of queued + // writers. + return l.queuedWriters.Len() == 0 } // assertEmptyLock asserts that the lockState is empty. This condition must hold @@ -1712,36 +1674,24 @@ func (l *lockState) tryActiveWait( } } - if l.reservation != nil { - if l.reservation == g { - // Already reserved by this request. + if !l.holder.locked && l.queuedWriters.Len() > 0 { + qg := l.queuedWriters.Front().Value.(*queuedGuard) + if qg.guard == g { + // Already claimed by this request. return false, false } - // A non-transactional write request never makes or breaks reservations, and - // only waits for a reservation if the reservation has a lower seqNum. Note - // that `str == lock.None && lockHolderTxn == nil` was already checked - // above. - if g.txn == nil && l.reservation.seqNum > g.seqNum { - // Reservation is held by a request with a higher seqNum and g is a - // non-transactional request. Ignore the reservation. + // A non-transactional write request never makes or breaks claims, and only + // waits for a claim if the claim holder has a lower seqNum. Note that `str + // == lock.None && lockHolderTxn == nil` was already checked above. + if g.txn == nil && qg.guard.seqNum > g.seqNum { + // Claimed by a request with a higher seqNum and g is a non-transactional + // request. Ignore the claim. return false, false } } // Incompatible with whoever is holding lock or reservation. - if l.reservation != nil && str == lock.Intent && l.tryBreakReservation(g.seqNum) { - l.reservation = g - g.mu.Lock() - g.mu.locks[l] = struct{}{} - g.mu.Unlock() - // There cannot be waitingReaders, since they do not wait for - // reservations. And the set of active queuedWriters has not changed, but - // they do need to be told about the change in who they are waiting for. - l.informActiveWaiters() - return false, false - } - waitForState := waitingState{ kind: waitFor, key: l.key, @@ -1814,7 +1764,7 @@ func (l *lockState) tryActiveWait( g.mu.locks[l] = struct{}{} waitForState.queuedWriters = l.queuedWriters.Len() // update field } - if replicatedLockFinalizedTxn != nil && l.queuedWriters.Front().Value.(*queuedGuard) == qg { + if (replicatedLockFinalizedTxn != nil || !l.holder.locked) && l.queuedWriters.Front().Value.(*queuedGuard) == qg { // First waiter, so should not wait. NB: this inactive waiter can be // non-transactional. qg.active = false @@ -1832,8 +1782,10 @@ func (l *lockState) tryActiveWait( } } if !wait { - g.toResolve = append( - g.toResolve, roachpb.MakeLockUpdate(replicatedLockFinalizedTxn, roachpb.Span{Key: l.key})) + if replicatedLockFinalizedTxn != nil { + g.toResolve = append( + g.toResolve, roachpb.MakeLockUpdate(replicatedLockFinalizedTxn, roachpb.Span{Key: l.key})) + } return false, false } // Make it an active waiter. @@ -1869,17 +1821,31 @@ func (l *lockState) isNonConflictingLock(g *lockTableGuardImpl, str lock.Strengt // Lock is not empty. lockHolderTxn, lockHolderTS := l.getLockHolder() if lockHolderTxn == nil { - // Reservation holders are non-conflicting. + // Transactions that have claimed the lock, but have not acquired it yet, + // are considered non-conflicting. + // + // Optimistic evaluation may call into this function with or without holding + // latches. It's worth considering both these cases separately: + // + // 1. If Optimistic evaluation is holding latches, then there cannot be a + // conflicting request that has claimed (but not acquired) the lock that is + // also holding latches. A request could have claimed this lock, discovered + // a different lock, and dropped its latches before waiting in this second + // lock's wait queue. In such cases, the request that claimed this lock will + // have to re-acquire and re-scan the lock table after this optimistic + // evaluation request drops its latches. // - // When optimistic evaluation holds latches, there cannot be a conflicting - // reservation holder that is also holding latches (reservation holder - // without latches can happen due to lock discovery). So after this - // optimistic evaluation succeeds and releases latches, the reservation - // holder will acquire latches and scan the lock table again. When - // optimistic evaluation does not hold latches, it will check for - // conflicting latches before declaring success and a reservation holder - // that holds latches will be discovered, and the optimistic evaluation - // will retry as pessimistic. + // 2. If optimistic evaluation does not hold latches, then it will check for + // conflicting latches before declaring success. A request that claimed this + // lock, did not discover any other locks, and proceeded to evaluation would + // thus conflict on latching with our request going through optimistic + // evaluation. This will be detected, and the request will have to retry + // pessimistically. + // + // All this is to say that if we found a claimed, but not yet acquired lock, + // we can treat it as non-conflicting. It'll either be detected as a true + // conflict when we check for conflicting latches, or the request that + // claimed the lock will know what happened and what to do about it. return true } if g.isSameTxn(lockHolderTxn) { @@ -2026,47 +1992,44 @@ func (l *lockState) acquireLock(acq *roachpb.LockAcquisition, clock *hlc.Clock) } return nil } - // Not already held, so may have been reserved by this request. There is also - // the possibility that some other request has broken this reservation because - // of a concurrent release but that is harmless since this request is - // holding latches and has proceeded to evaluation. - if l.reservation != nil { - if l.reservation.txn.ID != acq.Txn.ID { - // Reservation is broken. - qg := &queuedGuard{ - guard: l.reservation, - active: false, - } - l.queuedWriters.PushFront(qg) - } else { - // Else, reservation is not broken, or broken by a different request - // from the same transaction. In the latter case, both requests are not - // actively waiting at this lock. We don't know which is in the queue - // and which is holding the reservation but it does not matter. Both - // will have their requestGuardImpl.mu.locks updated and neither will be - // in the queue at the end of this method. - l.reservation.mu.Lock() - delete(l.reservation.mu.locks, l) - l.reservation.mu.Unlock() - } - if l.waitingReaders.Len() > 0 { - panic("lockTable bug") - } - } else { - if l.queuedWriters.Len() > 0 || l.waitingReaders.Len() > 0 { - panic("lockTable bug") - } + + // 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. + // + // [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. + + l.releaseWritersFromTxn(&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. + if l.waitingReaders.Len() > 0 { + panic("lockTable bug") } - l.reservation = nil + l.holder.locked = true l.holder.holder[acq.Durability].txn = &acq.Txn l.holder.holder[acq.Durability].ts = acq.Txn.WriteTimestamp l.holder.holder[acq.Durability].seqs = append([]enginepb.TxnSeq(nil), acq.Txn.Sequence) l.holder.startTime = clock.PhysicalTime() - // If there are waiting requests from the same txn, they no longer need to wait. - l.releaseWritersFromTxn(&acq.Txn) - // Inform active waiters since lock has transitioned to held. l.informActiveWaiters() return nil @@ -2106,20 +2069,6 @@ func (l *lockState) discoveredLock( holder.seqs = append(holder.seqs, txn.Sequence) } - // Queue the existing reservation holder. Note that this reservation - // holder may not be equal to g due to two reasons (a) the reservation - // of g could have been broken even though g is holding latches (see - // the comment in acquireLock()), (b) g may be a non-transactional - // request (read or write) that can ignore the reservation. - if l.reservation != nil { - qg := &queuedGuard{ - guard: l.reservation, - active: false, - } - l.queuedWriters.PushFront(qg) - l.reservation = nil - } - switch accessStrength { case lock.None: // Don't enter the lock's queuedReaders list, because all queued readers @@ -2217,15 +2166,6 @@ func (l *lockState) tryClearLock(force bool) bool { } l.clearLockHolder() - // Clear reservation. - if l.reservation != nil { - g := l.reservation - g.mu.Lock() - delete(g.mu.locks, l) - g.mu.Unlock() - l.reservation = nil - } - // Clear waitingReaders. for e := l.waitingReaders.Front(); e != nil; { g := e.Value.(*lockTableGuardImpl) @@ -2431,11 +2371,12 @@ func (l *lockState) removeReader(e *list.Element) bool { return false } -// A request known to this lockState is done. The request could be a reserver, -// or waiting reader or writer. Acquires l.mu. 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 lockState can be garbage collected. +// A request known to this lockState 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 lockState can be garbage +// collected. +// // Acquires l.mu. func (l *lockState) requestDone(g *lockTableGuardImpl) (gc bool) { l.mu.Lock() @@ -2449,29 +2390,31 @@ func (l *lockState) requestDone(g *lockTableGuardImpl) (gc bool) { delete(g.mu.locks, l) g.mu.Unlock() - doneRemoval := false - if l.reservation == g { - l.reservation = nil - l.maybeReleaseFirstTransactionalWriter() - doneRemoval = true - } - // May be in queuedWriters or waitingReaders. distinguishedRemoved := false - if !doneRemoval { - for e := l.queuedWriters.Front(); e != nil; e = e.Next() { - qg := e.Value.(*queuedGuard) - if qg.guard == g { - l.queuedWriters.Remove(e) - if qg.guard == l.distinguishedWaiter { - distinguishedRemoved = true - l.distinguishedWaiter = nil - } - doneRemoval = true - break + doneRemoval := false + for e := l.queuedWriters.Front(); e != nil; e = e.Next() { + qg := e.Value.(*queuedGuard) + if qg.guard == g { + l.queuedWriters.Remove(e) + if qg.guard == l.distinguishedWaiter { + distinguishedRemoved = true + l.distinguishedWaiter = nil } + doneRemoval = true + break } } + + if !l.holder.locked && 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. + l.maybeReleaseFirstTransactionalWriter() + } + if !doneRemoval { for e := l.waitingReaders.Front(); e != nil; e = e.Next() { gg := e.Value.(*lockTableGuardImpl) @@ -2535,17 +2478,15 @@ func (l *lockState) tryFreeLockOnReplicatedAcquire() bool { return true } -// The lock has transitioned from locked to unlocked. There could be waiters, -// but there cannot be a reservation. +// The lock has transitioned from locked to unlocked. There could be waiters. // // REQUIRES: l.mu is locked. +// TODO(arul): rename this + improve comment here to better reflect the state +// transitions this function performs. func (l *lockState) lockIsFree() (gc bool) { if l.holder.locked { panic("called lockIsFree on lock with holder") } - if l.reservation != nil { - panic("called lockIsFree on lock with reservation") - } // All waiting readers don't need to wait here anymore. // NB: all waiting readers are by definition active waiters. @@ -2558,8 +2499,8 @@ func (l *lockState) lockIsFree() (gc bool) { l.maybeReleaseFirstTransactionalWriter() // We've already cleared waiting readers above. The lock can be released if - // there is no reservation or waiting writers. - if l.queuedWriters.Len() == 0 && l.reservation == nil { + // there are no waiting writers, active or otherwise. + if l.queuedWriters.Len() == 0 { l.assertEmptyLock() return true } @@ -2608,27 +2549,24 @@ func (l *lockState) maybeReleaseFirstTransactionalWriter() { return // no transactional writer } - // First waiting writer (it must be transactional) gets the reservation. + // 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 := l.queuedWriters.Front() qg := e.Value.(*queuedGuard) g := qg.guard - l.reservation = g - // TODO(arul): Even though we're removing this writer from the list, we do so - // directly without calling into removeWriter. This is because we don't want - // to modify the g.mu.locks bookkeeping when giving this request the - // reservation. This is temporary -- once we remove the concept of - // reservations, we'll no longer be removing the request from the - // queuedWriters list. - l.queuedWriters.Remove(e) if qg.active { + qg.active = false // mark as inactive if g == l.distinguishedWaiter { + // We're only clearing the distinguishedWaiter for now; a new one will be + // selected below in the call to informActiveWaiters. l.distinguishedWaiter = nil } g.mu.Lock() g.doneActivelyWaitingAtLock() g.mu.Unlock() } - // Else inactive waiter and is waiting elsewhere. + // Else the waiter is already inactive. // Tell the active waiters who they are waiting for. l.informActiveWaiters() diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index a1cde4a54d6c..9e1ed8df3398 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -170,9 +170,9 @@ func (w *lockTableWaiterImpl) WaitOn( // immediately without waiting. If the conflict is a lock then // push the lock holder's transaction using a PUSH_TOUCH to // determine whether the lock is abandoned or whether its holder - // is still active. If the conflict is a reservation holder, - // raise an error immediately, we know the reservation holder is - // active. + // is still active. If the conflict is a concurrent transaction that + // is being sequence through the lock table that has claimed the lock, + // raise an error immediately -- we know the request is active. if state.held { err = w.pushLockTxn(ctx, req, state) } else { @@ -203,10 +203,10 @@ func (w *lockTableWaiterImpl) WaitOn( livenessPush := state.kind == waitForDistinguished deadlockPush := true - // If the conflict is a reservation holder and not a held lock then - // there's no need to perform a liveness push - the request must be - // alive or its context would have been canceled and it would have - // exited its lock wait-queues. + // If the conflict is a claimant transaction that hasn't acquired the + // lock yet there's no need to perform a liveness push - the request + // must be alive or its context would have been canceled and it would + // have exited its lock wait-queues. if !state.held { livenessPush = false } @@ -214,7 +214,7 @@ func (w *lockTableWaiterImpl) WaitOn( // For non-transactional requests, there's no need to perform // deadlock detection because a non-transactional request can // not be part of a dependency cycle. Non-transactional requests - // cannot hold locks or reservations. + // cannot hold locks (and by extension, claim them). if req.Txn == nil { deadlockPush = false } @@ -310,10 +310,10 @@ func (w *lockTableWaiterImpl) WaitOn( return w.pushLockTxn(ctx, req, state) case waitSelf: - // Another request from the same transaction is the reservation - // holder of this lock wait-queue. This can only happen when the - // request's transaction is sending multiple requests concurrently. - // Proceed with waiting without pushing anyone. + // Another request from the same transaction has claimed the lock (but + // not yet acquired it). This can only happen when the request's + // transaction is sending multiple requests concurrently. Proceed with + // waiting without pushing anyone. case waitQueueMaxLengthExceeded: // The request attempted to wait in a lock wait-queue whose length was @@ -368,14 +368,14 @@ func (w *lockTableWaiterImpl) WaitOn( // through intent resolution. The request has a dependency on the // entire conflicting transaction. // - // However, if the request is conflicting with another request (a - // reservation holder) then it pushes the reservation holder - // asynchronously while continuing to listen to state transition in - // the lockTable. This allows the request to cancel its push if the - // conflicting reservation exits the lock wait-queue without leaving - // behind a lock. In this case, the request has a dependency on the - // conflicting request but not necessarily the entire conflicting - // transaction. + // However, if the request is conflicting with another request (that has + // claimed the lock, but not yet acquired it) then it pushes the + // claimant transaction asynchronously while continuing to listen to + // state transition in the lockTable. This allows the request to cancel + // its push if the conflicting claimant transaction exits the lock + // wait-queue without leaving behind a lock. In this case, the request + // has a dependency on the conflicting request but not necessarily the + // entire conflicting transaction. if timerWaitingState.held { return w.pushLockTxn(ctx, req, timerWaitingState) } @@ -407,8 +407,9 @@ func (w *lockTableWaiterImpl) WaitOn( // Resolve the conflict without waiting. If the conflict is a lock // then push the lock holder's transaction using a PUSH_TOUCH to // determine whether the lock is abandoned or whether its holder is - // still active. If the conflict is a reservation holder, raise an - // error immediately, we know the reservation holder is active. + // still active. If the conflict is a claimant transaction, raise an + // error immediately, we know the transaction that has claimed (but not + // yet acquired) the lock active. if timerWaitingState.held { return w.pushLockTxnAfterTimeout(ctx, req, timerWaitingState) } @@ -707,7 +708,7 @@ func (w *lockTableWaiterImpl) pushRequestTxn( // Even if the push succeeded and aborted the other transaction to break a // deadlock, there's nothing for the pusher to clean up. The conflicting - // request will quickly exit the lock wait-queue and release its reservation + // request will quickly exit the lock wait-queue and release its claim // once it notices that it is aborted and the pusher will be free to proceed // because it was not waiting on any locks. If the pusher's request does end // up hitting a lock which the pushee fails to clean up, it will perform the @@ -744,16 +745,15 @@ func (w *lockTableWaiterImpl) pushRequestTxn( // Example: // // req(1, txn1), req(1, txn2) are both waiting on a lock held by txn3, and - // they respectively hold a reservation on key "a" and key "b". req(2, txn2) - // queues up behind the reservation on key "a" and req(2, txn1) queues up - // behind the reservation on key "b". Now the dependency cycle between txn1 - // and txn2 only involves requests, but some of the requests here also - // depend on a lock. So when both txn1, txn2 are aborted, the req(1, txn1), - // req(1, txn2) are guaranteed to eventually notice through self-directed - // QueryTxn requests and will exit the lockTable, allowing req(2, txn1) and - // req(2, txn2) to get the reservation and now they no longer depend on each - // other. - // + // they respectively hold a claim (but not the lock itself) on key "a" and + // key "b". req(2, txn2) queues up behind the claim on key "a" and req(2, + // txn1) queues up behind the claim on key "b". Now the dependency cycle + // between txn1 and txn2 only involves requests, but some of the requests + // here also depend on a lock. So when both txn1, txn2 are aborted, the + // req(1, txn1), req(1, txn2) are guaranteed to eventually notice through + // self-directed QueryTxn requests and will exit the lockTable, allowing + // req(2, txn1) and req(2, txn2) to claim the lock and now they no longer + // depend on each other. return nil } @@ -970,7 +970,7 @@ const tagWaitKey = "wait_key" const tagWaitStart = "wait_start" // tagLockHolderTxn is the tracing span tag indicating the ID of the txn holding -// the lock (or a reservation on the lock) that the request is currently waiting +// the lock (or has claimed the lock) that the request is currently waiting // on. const tagLockHolderTxn = "holder_txn" diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 5d3003ccdf23..cd704319ca68 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -337,8 +337,9 @@ func testWaitPush(t *testing.T, k waitKind, makeReq func() Request, expPushTS hl return } - // Non-transactional requests without a timeout do not push - // reservations, only locks. They wait for doneWaiting. + // Non-transactional requests without a timeout do not push transactions + // that have claimed a lock; they only push lock holders. Instead, they + // wait for doneWaiting. if req.Txn == nil && !lockHeld { defer notifyUntilDone(t, g)() err := w.WaitOn(ctx, req, g) @@ -498,7 +499,7 @@ func testErrorWaitPush( testutils.RunTrueAndFalse(t, "lockHeld", func(t *testing.T, lockHeld bool) { testutils.RunTrueAndFalse(t, "pusheeActive", func(t *testing.T, pusheeActive bool) { if !lockHeld && !pusheeActive { - // !lockHeld means a lock reservation, so is only possible when + // !lockHeld means a lock claim, so is only possible when // pusheeActive is true. skip.IgnoreLint(t, "incompatible params") } @@ -658,7 +659,7 @@ func testWaitPushWithTimeout(t *testing.T, k waitKind, makeReq func() Request) { skip.IgnoreLint(t, "incompatible params") } if !lockHeld && !pusheeActive { - // !lockHeld means a lock reservation, so is only possible when + // !lockHeld means a lock is claimed, so is only possible when // pusheeActive is true. skip.IgnoreLint(t, "incompatible params") } diff --git a/pkg/kv/kvserver/concurrency/metrics.go b/pkg/kv/kvserver/concurrency/metrics.go index efb09e544d27..b22b76a71a15 100644 --- a/pkg/kv/kvserver/concurrency/metrics.go +++ b/pkg/kv/kvserver/concurrency/metrics.go @@ -32,6 +32,8 @@ type LockTableMetrics struct { // marked as held. TotalLockHoldDurationNanos int64 // The number of locks not held, but with reservations. + // TODO(arul): this needs to be fixed now that we don't have reservations + // anymore. See https://github.com/cockroachdb/cockroach/issues/103894. LocksWithReservation int64 // The number of locks with non-empty wait-queues. LocksWithWaitQueues int64 @@ -62,9 +64,10 @@ type LockTableMetrics struct { type LockMetrics struct { // The lock's key. Key roachpb.Key - // Is the lock actively held by a transaction, or just a reservation? + // Is the lock actively held by a transaction, or has been just claimed. Held bool - // The number of nanoseconds this lock has been in the lock table and marked as held. + // The number of nanoseconds this lock has been in the lock table and marked + // as held. HoldDurationNanos int64 // The number of waiters in the lock's wait queue. Waiters int64 @@ -72,9 +75,11 @@ type LockMetrics struct { WaitingReaders int64 // The number of waiting writers in the lock's wait queue. WaitingWriters int64 - // The total number of nanoseconds all waiters have been in the lock's wait queue. + // The total number of nanoseconds all waiters have been in the lock's wait + // queue. WaitDurationNanos int64 - // The maximum number of nanoseconds a waiter has been in the lock's wait queue. + // The maximum number of nanoseconds a waiter has been in the lock's wait + // queue. MaxWaitDurationNanos int64 } 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 66583bd46453..7d92ae0fdea2 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents @@ -201,7 +201,8 @@ debug-lock-table ---- num=2 lock: "a" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 lock: "b" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] queued writers: @@ -229,9 +230,11 @@ debug-lock-table ---- num=3 lock: "a" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 lock: "b" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 lock: "c" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] queued writers: @@ -254,11 +257,14 @@ debug-lock-table ---- num=3 lock: "a" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 lock: "b" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 lock: "c" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 finish req=req1 ---- @@ -612,7 +618,8 @@ num=4 req: 8, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 8 lock: "c" - res: req: 5, txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,1, seq: 0 + queued writers: + active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "d" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] queued writers: @@ -644,9 +651,11 @@ debug-lock-table ---- num=3 lock: "c" - res: req: 5, txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,1, seq: 0 + queued writers: + active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "d" - res: req: 5, txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,1, seq: 0 + queued writers: + active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "e" holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks index 90e48375defd..d249a2535bd8 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks @@ -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: 1, queuedReaders: 0) +[7] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"a"› (queuedWriters: 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 @@ -575,7 +575,8 @@ num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - res: req: 17, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 17, txn: 00000004-0000-0000-0000-000000000000 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -600,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: 1, queuedReaders: 0) +[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"b"› (queuedWriters: 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 @@ -628,8 +629,8 @@ num=3 active: true req: 19, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 19 lock: "b" - res: req: 17, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: + active: false req: 17, txn: 00000004-0000-0000-0000-000000000000 active: true req: 18, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 18 lock: "c" @@ -810,7 +811,8 @@ num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - res: req: 23, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 23, txn: 00000004-0000-0000-0000-000000000000 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -835,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: 1, queuedReaders: 0) +[5] sequence req1w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"b"› (queuedWriters: 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 @@ -863,8 +865,8 @@ num=3 active: true req: 25, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 25 lock: "b" - res: req: 23, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: + active: false req: 23, txn: 00000004-0000-0000-0000-000000000000 active: true req: 24, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 24 lock: "c" @@ -1072,7 +1074,7 @@ on-txn-updated txn=txn2 status=committed [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: 1, queuedReaders: 0) +[5] sequence req4w: lock wait-queue event: wait for (distinguished) txn 00000005 running request @ key ‹"b"› (queuedWriters: 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 @@ -1082,10 +1084,11 @@ debug-lock-table ---- num=3 lock: "a" - res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 30, txn: 00000004-0000-0000-0000-000000000000 lock: "b" - res: req: 29, txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: + active: false req: 29, txn: 00000005-0000-0000-0000-000000000000 active: true req: 30, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 30 lock: "c" @@ -1110,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: 1, queuedReaders: 0) +[6] sequence req3w2: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"a"› (queuedWriters: 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 @@ -1120,13 +1123,13 @@ debug-lock-table ---- num=3 lock: "a" - res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: + active: false req: 30, txn: 00000004-0000-0000-0000-000000000000 active: true req: 31, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 31 lock: "b" - res: req: 29, txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: + active: false req: 29, txn: 00000005-0000-0000-0000-000000000000 active: true req: 30, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 30 lock: "c" diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout index 5744a3c495b5..1812eac9491c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout @@ -140,7 +140,8 @@ debug-lock-table ---- num=2 lock: "k2" - res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 + queued writers: + active: false req: 3, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -163,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: 1, queuedReaders: 0) +[6] sequence reqTimeout2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k2"› (queuedWriters: 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 intents on ‹"k2"› [reason=lock_timeout] @@ -206,7 +207,8 @@ debug-lock-table ---- num=3 lock: "k2" - res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 + queued writers: + active: false req: 3, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority index ad6644c5b2e1..45d58210fa2f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority @@ -231,7 +231,7 @@ sequence req=req7 [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: 1, queuedReaders: 0) +[7] sequence req7: lock wait-queue event: wait for (distinguished) txn 00000004 running request @ key ‹"kLow2"› (queuedWriters: 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 @@ -377,7 +377,7 @@ sequence req=req11 [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: 1, queuedReaders: 0) +[11] sequence req11: lock wait-queue event: wait for (distinguished) txn 00000007 running request @ key ‹"kNormal2"› (queuedWriters: 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 @@ -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: 1, queuedReaders: 0) +[15] sequence req15: lock wait-queue event: wait for (distinguished) txn 00000008 running request @ key ‹"kHigh2"› (queuedWriters: 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 56257c38e7bc..a91d47f5d5c9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded @@ -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: 2, queuedReaders: 0) +[3] sequence req3: lock wait-queue event: wait for (distinguished) txn 00000002 running request @ key ‹"k"› (queuedWriters: 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: 2, queuedReaders: 0) +[4] sequence req4: lock wait-queue event: wait for txn 00000002 running request @ key ‹"k"› (queuedWriters: 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 @@ -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: 1, queuedReaders: 0) +[4] sequence req4: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k"› (queuedWriters: 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 a36674265324..c35c14595e00 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener @@ -212,7 +212,8 @@ debug-lock-table ---- num=1 lock: "k" - res: req: 3, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 3, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k ---- @@ -321,7 +322,8 @@ debug-lock-table ---- num=1 lock: "k" - res: req: 4, txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 4, txn: 00000003-0000-0000-0000-000000000000 on-lock-acquired req=req3 key=k ---- @@ -478,7 +480,8 @@ debug-lock-table ---- num=1 lock: "k" - res: req: 6, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 6, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k ---- @@ -697,7 +700,8 @@ debug-lock-table ---- num=1 lock: "k" - res: req: 10, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 10, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k ---- @@ -854,7 +858,8 @@ debug-lock-table ---- num=1 lock: "k" - res: req: 12, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 12, txn: 00000002-0000-0000-0000-000000000000 on-lock-acquired req=req2 key=k ---- 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 faca22302ce4..b3475e946488 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error @@ -142,7 +142,8 @@ debug-lock-table ---- num=2 lock: "k2" - res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 + queued writers: + active: false req: 3, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -165,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: 1, queuedReaders: 0) +[6] sequence reqNoWait2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k2"› (queuedWriters: 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 intents on ‹"k2"› [reason=wait_policy] @@ -206,7 +207,8 @@ debug-lock-table ---- num=3 lock: "k2" - res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 + queued writers: + active: false req: 3, txn: 00000003-0000-0000-0000-000000000000 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: 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 647769e9b232..aac2d36f7aa1 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip @@ -113,7 +113,8 @@ num=4 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "k4" - res: req: 4, txn: 00000004-0000-0000-0000-000000000000, ts: 13.000000000,0, seq: 0 + queued writers: + active: false req: 4, txn: 00000004-0000-0000-0000-000000000000 # ------------------------------------------------------------- # Read-only request with WaitPolicy_Skip hits lock sequences diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self index 884cc0453cb2..e446c348a0a8 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self @@ -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: 2, queuedReaders: 0) +[3] sequence reqTxnMiddle: lock wait-queue event: wait for (distinguished) txn 00000001 running request @ key ‹"k"› (queuedWriters: 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,8 +101,8 @@ debug-lock-table ---- num=1 lock: "k" - res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: + active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 active: true req: 4, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 3 @@ -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: 1, queuedReaders: 0) +[4] sequence reqTxn2: lock wait-queue event: wait for (distinguished) txn 00000003 running request @ key ‹"k"› (queuedWriters: 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/add_discovered b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered index ecb102013a38..cd77f1216895 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered @@ -68,8 +68,8 @@ release txn=txn1 span=a ---- num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, seq: 0 queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 @@ -118,7 +118,8 @@ release txn=txn3 span=a ---- num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, 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 ceb44fce612f..1ce7eff46680 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic @@ -135,7 +135,8 @@ update txn=txn1 ts=11,1 epoch=1 span=b ---- num=3 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" @@ -156,9 +157,11 @@ update txn=txn1 ts=11,1 epoch=1 span=c,e ---- num=3 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] @@ -188,9 +191,11 @@ num=4 queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] @@ -202,9 +207,11 @@ num=5 queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" @@ -252,9 +259,11 @@ num=5 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" @@ -274,11 +283,11 @@ locks: 5 locksheld: 3 totallockholddurationnanos: 11400000000 lockswithreservation: 2 -lockswithwaitqueues: 1 -waiters: 1 +lockswithwaitqueues: 3 +waiters: 3 waitingreaders: 0 -waitingwriters: 1 -totalwaitdurationnanos: 2000000000 +waitingwriters: 3 +totalwaitdurationnanos: 6000000000 topklocksbywaiters: - key: - 97 @@ -289,22 +298,24 @@ topklocksbywaiters: waitingwriters: 1 waitdurationnanos: 2000000000 maxwaitdurationnanos: 2000000000 -- key: [] +- key: + - 98 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 -- key: [] + waitingwriters: 1 + waitdurationnanos: 2000000000 + maxwaitdurationnanos: 2000000000 +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 1 + waitdurationnanos: 2000000000 + maxwaitdurationnanos: 2000000000 topklocksbyholdduration: - key: - 101 @@ -343,22 +354,24 @@ topklocksbywaitduration: waitingwriters: 1 waitdurationnanos: 2000000000 maxwaitdurationnanos: 2000000000 -- key: [] +- key: + - 98 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 -- key: [] + waitingwriters: 1 + waitdurationnanos: 2000000000 + maxwaitdurationnanos: 2000000000 +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 1 + waitdurationnanos: 2000000000 + maxwaitdurationnanos: 2000000000 # 300ms passes before req5 time-tick ms=300 @@ -384,9 +397,11 @@ num=5 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" @@ -430,12 +445,13 @@ num=5 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" @@ -447,12 +463,21 @@ locks: 5 locksheld: 3 totallockholddurationnanos: 12600000000 lockswithreservation: 2 -lockswithwaitqueues: 2 -waiters: 2 +lockswithwaitqueues: 3 +waiters: 4 waitingreaders: 0 -waitingwriters: 2 -totalwaitdurationnanos: 2400000000 +waitingwriters: 4 +totalwaitdurationnanos: 7200000000 topklocksbywaiters: +- key: + - 98 + held: false + holddurationnanos: 0 + waiters: 2 + waitingreaders: 0 + waitingwriters: 2 + waitdurationnanos: 2400000000 + maxwaitdurationnanos: 2400000000 - key: - 97 held: true @@ -463,22 +488,14 @@ topklocksbywaiters: waitdurationnanos: 2400000000 maxwaitdurationnanos: 2400000000 - key: - - 98 + - 99 held: false holddurationnanos: 0 waiters: 1 waitingreaders: 0 waitingwriters: 1 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 -- key: [] - held: false - holddurationnanos: 0 - waiters: 0 - waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitdurationnanos: 2400000000 + maxwaitdurationnanos: 2400000000 topklocksbyholdduration: - key: - 101 @@ -521,19 +538,20 @@ topklocksbywaitduration: - 98 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 -- key: [] + waitingwriters: 2 + waitdurationnanos: 2400000000 + maxwaitdurationnanos: 2400000000 +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 1 + waitdurationnanos: 2400000000 + maxwaitdurationnanos: 2400000000 guard-state r=req6 ---- @@ -582,13 +600,13 @@ num=5 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -603,38 +621,38 @@ locksheld: 3 totallockholddurationnanos: 13350000000 lockswithreservation: 2 lockswithwaitqueues: 3 -waiters: 3 +waiters: 5 waitingreaders: 0 -waitingwriters: 3 -totalwaitdurationnanos: 2900000000 +waitingwriters: 5 +totalwaitdurationnanos: 8200000000 topklocksbywaiters: -- key: - - 97 - held: true - holddurationnanos: 2650000000 - waiters: 1 - waitingreaders: 0 - waitingwriters: 1 - waitdurationnanos: 2650000000 - maxwaitdurationnanos: 2650000000 - key: - 98 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 - waitdurationnanos: 250000000 - maxwaitdurationnanos: 250000000 + waitingwriters: 2 + waitdurationnanos: 2900000000 + maxwaitdurationnanos: 2650000000 - key: - 99 held: false holddurationnanos: 0 + waiters: 2 + waitingreaders: 0 + waitingwriters: 2 + waitdurationnanos: 2650000000 + maxwaitdurationnanos: 2650000000 +- key: + - 97 + held: true + holddurationnanos: 2650000000 waiters: 1 waitingreaders: 0 waitingwriters: 1 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitdurationnanos: 2650000000 + maxwaitdurationnanos: 2650000000 topklocksbyholdduration: - key: - 101 @@ -677,20 +695,20 @@ topklocksbywaitduration: - 98 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 - waitdurationnanos: 250000000 - maxwaitdurationnanos: 250000000 + waitingwriters: 2 + waitdurationnanos: 2900000000 + maxwaitdurationnanos: 2650000000 - key: - 99 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 2 + waitdurationnanos: 2650000000 + maxwaitdurationnanos: 2650000000 query @@ -719,15 +737,16 @@ release txn=txn3 span=a ---- num=5 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -747,15 +766,16 @@ print ---- num=5 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -772,37 +792,37 @@ locks: 5 locksheld: 2 totallockholddurationnanos: 10900000000 lockswithreservation: 3 -lockswithwaitqueues: 3 -waiters: 3 +lockswithwaitqueues: 4 +waiters: 6 waitingreaders: 1 -waitingwriters: 2 +waitingwriters: 5 totalwaitdurationnanos: 450000000 topklocksbywaiters: - key: - 98 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 + waitingwriters: 2 waitdurationnanos: 350000000 maxwaitdurationnanos: 350000000 - key: - 99 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 + waitingwriters: 2 waitdurationnanos: 100000000 maxwaitdurationnanos: 100000000 - key: - - 102 - held: true - holddurationnanos: 2750000000 + - 97 + held: false + holddurationnanos: 0 waiters: 1 - waitingreaders: 1 - waitingwriters: 0 + waitingreaders: 0 + waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 topklocksbyholdduration: @@ -837,27 +857,27 @@ topklocksbywaitduration: - 98 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 + waitingwriters: 2 waitdurationnanos: 350000000 maxwaitdurationnanos: 350000000 - key: - 99 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 + waitingwriters: 2 waitdurationnanos: 100000000 maxwaitdurationnanos: 100000000 - key: - - 102 - held: true - holddurationnanos: 2750000000 + - 97 + held: false + holddurationnanos: 0 waiters: 1 - waitingreaders: 1 - waitingwriters: 0 + waitingreaders: 0 + waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 @@ -889,15 +909,16 @@ release txn=txn3 span=f ---- num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -931,15 +952,16 @@ acquire r=req4 k=b durability=r ---- num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 lock: "e" @@ -949,7 +971,8 @@ acquire r=req4 k=c durability=r ---- num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: @@ -975,7 +998,8 @@ print ---- num=4 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: @@ -995,12 +1019,21 @@ locks: 4 locksheld: 3 totallockholddurationnanos: 8650000000 lockswithreservation: 1 -lockswithwaitqueues: 2 -waiters: 2 +lockswithwaitqueues: 3 +waiters: 3 waitingreaders: 0 -waitingwriters: 2 -totalwaitdurationnanos: 1450000000 +waitingwriters: 3 +totalwaitdurationnanos: 1950000000 topklocksbywaiters: +- key: + - 97 + held: false + holddurationnanos: 0 + waiters: 1 + waitingreaders: 0 + waitingwriters: 1 + waitdurationnanos: 500000000 + maxwaitdurationnanos: 500000000 - key: - 98 held: true @@ -1019,14 +1052,6 @@ topklocksbywaiters: waitingwriters: 1 waitdurationnanos: 600000000 maxwaitdurationnanos: 600000000 -- key: [] - held: false - holddurationnanos: 0 - waiters: 0 - waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 topklocksbyholdduration: - key: - 101 @@ -1074,14 +1099,15 @@ topklocksbywaitduration: waitingwriters: 1 waitdurationnanos: 600000000 maxwaitdurationnanos: 600000000 -- key: [] +- key: + - 97 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 1 + waitdurationnanos: 500000000 + maxwaitdurationnanos: 500000000 dequeue r=req4 ---- @@ -1128,7 +1154,8 @@ num=3 active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] @@ -1149,7 +1176,8 @@ num=3 active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] @@ -1159,11 +1187,11 @@ locks: 3 locksheld: 2 totallockholddurationnanos: 12650000000 lockswithreservation: 1 -lockswithwaitqueues: 1 -waiters: 1 +lockswithwaitqueues: 2 +waiters: 2 waitingreaders: 0 -waitingwriters: 1 -totalwaitdurationnanos: 2850000000 +waitingwriters: 2 +totalwaitdurationnanos: 5450000000 topklocksbywaiters: - key: - 98 @@ -1174,14 +1202,15 @@ topklocksbywaiters: waitingwriters: 1 waitdurationnanos: 2850000000 maxwaitdurationnanos: 2850000000 -- key: [] +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 1 + waitdurationnanos: 2600000000 + maxwaitdurationnanos: 2600000000 - key: [] held: false holddurationnanos: 0 @@ -1227,14 +1256,15 @@ topklocksbywaitduration: waitingwriters: 1 waitdurationnanos: 2850000000 maxwaitdurationnanos: 2850000000 -- key: [] +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 1 + waitdurationnanos: 2600000000 + maxwaitdurationnanos: 2600000000 - key: [] held: false holddurationnanos: 0 @@ -1256,9 +1286,11 @@ release txn=txn2 span=b ---- num=3 lock: "b" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] @@ -1290,10 +1322,11 @@ print ---- num=3 lock: "b" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 active: false req: 7, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] @@ -1304,29 +1337,30 @@ locks: 3 locksheld: 1 totallockholddurationnanos: 10690000000 lockswithreservation: 2 -lockswithwaitqueues: 1 -waiters: 1 +lockswithwaitqueues: 2 +waiters: 3 waitingreaders: 0 -waitingwriters: 1 -totalwaitdurationnanos: 2640000000 +waitingwriters: 3 +totalwaitdurationnanos: 8420000000 topklocksbywaiters: - key: - 99 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 - waitdurationnanos: 2640000000 - maxwaitdurationnanos: 2640000000 -- key: [] + waitingwriters: 2 + waitdurationnanos: 5530000000 + maxwaitdurationnanos: 2890000000 +- key: + - 98 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 1 + waitdurationnanos: 2890000000 + maxwaitdurationnanos: 2890000000 - key: [] held: false holddurationnanos: 0 @@ -1363,22 +1397,23 @@ topklocksbyholdduration: maxwaitdurationnanos: 0 topklocksbywaitduration: - key: - - 99 + - 98 held: false holddurationnanos: 0 waiters: 1 waitingreaders: 0 waitingwriters: 1 - waitdurationnanos: 2640000000 - maxwaitdurationnanos: 2640000000 -- key: [] + waitdurationnanos: 2890000000 + maxwaitdurationnanos: 2890000000 +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 2 waitingreaders: 0 - waitingwriters: 0 - waitdurationnanos: 0 - maxwaitdurationnanos: 0 + waitingwriters: 2 + waitdurationnanos: 5530000000 + maxwaitdurationnanos: 2890000000 - key: [] held: false holddurationnanos: 0 @@ -1405,7 +1440,8 @@ dequeue r=req6 ---- num=2 lock: "c" - res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] @@ -1801,8 +1837,8 @@ release txn=txn1 span=c ---- num=1 lock: "c" - res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 10, txn: 00000000-0000-0000-0000-000000000002 active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 @@ -1827,8 +1863,8 @@ print ---- num=1 lock: "c" - res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: + active: false req: 10, txn: 00000000-0000-0000-0000-000000000002 active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 @@ -1840,18 +1876,18 @@ locksheld: 0 totallockholddurationnanos: 0 lockswithreservation: 1 lockswithwaitqueues: 1 -waiters: 2 +waiters: 3 waitingreaders: 0 -waitingwriters: 2 +waitingwriters: 3 totalwaitdurationnanos: 0 topklocksbywaiters: - key: - 99 held: false holddurationnanos: 0 - waiters: 2 + waiters: 3 waitingreaders: 0 - waitingwriters: 2 + waitingwriters: 3 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -1900,9 +1936,9 @@ topklocksbywaitduration: - 99 held: false holddurationnanos: 0 - waiters: 2 + waiters: 3 waitingreaders: 0 - waitingwriters: 2 + waitingwriters: 3 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -2074,7 +2110,8 @@ release txn=txn2 span=b,d ---- num=1 lock: "c" - res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 + queued writers: + active: false req: 11, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req11 ---- @@ -2084,7 +2121,8 @@ print ---- num=1 lock: "c" - res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 + queued writers: + active: false req: 11, txn: 00000000-0000-0000-0000-000000000003 metrics ---- @@ -2092,18 +2130,19 @@ locks: 1 locksheld: 0 totallockholddurationnanos: 0 lockswithreservation: 1 -lockswithwaitqueues: 0 -waiters: 0 +lockswithwaitqueues: 1 +waiters: 1 waitingreaders: 0 -waitingwriters: 0 +waitingwriters: 1 totalwaitdurationnanos: 0 topklocksbywaiters: -- key: [] +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 + waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -2148,12 +2187,13 @@ topklocksbyholdduration: waitdurationnanos: 0 maxwaitdurationnanos: 0 topklocksbywaitduration: -- key: [] +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 + waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -2211,13 +2251,15 @@ release txn=txn2 span=b,d ---- num=1 lock: "c" - res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, seq: 0 + queued writers: + active: false req: 14, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req15 ---- num=1 lock: "c" - res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, seq: 0 + queued writers: + active: false req: 14, txn: 00000000-0000-0000-0000-000000000001 new-request r=req16 txn=none ts=10,12 spans=none@c ---- @@ -2483,7 +2525,8 @@ release txn=txn1 span=c ---- num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 18, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: @@ -2498,7 +2541,8 @@ print ---- num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 18, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: @@ -2512,10 +2556,10 @@ locks: 2 locksheld: 1 totallockholddurationnanos: 0 lockswithreservation: 1 -lockswithwaitqueues: 1 -waiters: 2 +lockswithwaitqueues: 2 +waiters: 3 waitingreaders: 0 -waitingwriters: 2 +waitingwriters: 3 totalwaitdurationnanos: 0 topklocksbywaiters: - key: @@ -2527,12 +2571,13 @@ topklocksbywaiters: waitingwriters: 2 waitdurationnanos: 0 maxwaitdurationnanos: 0 -- key: [] +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 + waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -2571,20 +2616,21 @@ topklocksbyholdduration: maxwaitdurationnanos: 0 topklocksbywaitduration: - key: - - 100 - held: true + - 99 + held: false holddurationnanos: 0 - waiters: 2 + waiters: 1 waitingreaders: 0 - waitingwriters: 2 + waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 -- key: [] - held: false +- key: + - 100 + held: true holddurationnanos: 0 - waiters: 0 + waiters: 2 waitingreaders: 0 - waitingwriters: 0 + waitingwriters: 2 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -2600,10 +2646,11 @@ release txn=txn1 span=d ---- num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 18, txn: 00000000-0000-0000-0000-000000000002 lock: "d" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 18, txn: 00000000-0000-0000-0000-000000000002 active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 scan r=req18 @@ -2614,7 +2661,8 @@ acquire r=req18 k=d durability=u ---- num=2 lock: "c" - res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 18, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -2812,7 +2860,8 @@ num=2 active: true req: 22, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 22 lock: "d" - res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 23, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req23 ---- @@ -2831,9 +2880,11 @@ release txn=txn1 span=c ---- num=2 lock: "c" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 22, txn: 00000000-0000-0000-0000-000000000002 lock: "d" - res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 23, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req22 ---- @@ -2843,10 +2894,11 @@ print ---- num=2 lock: "c" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 22, txn: 00000000-0000-0000-0000-000000000002 lock: "d" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 22, txn: 00000000-0000-0000-0000-000000000002 active: false req: 23, txn: 00000000-0000-0000-0000-000000000003 metrics @@ -2855,27 +2907,28 @@ locks: 2 locksheld: 0 totallockholddurationnanos: 0 lockswithreservation: 2 -lockswithwaitqueues: 1 -waiters: 1 +lockswithwaitqueues: 2 +waiters: 3 waitingreaders: 0 -waitingwriters: 1 +waitingwriters: 3 totalwaitdurationnanos: 0 topklocksbywaiters: - key: - 100 held: false holddurationnanos: 0 - waiters: 1 + waiters: 2 waitingreaders: 0 - waitingwriters: 1 + waitingwriters: 2 waitdurationnanos: 0 maxwaitdurationnanos: 0 -- key: [] +- key: + - 99 held: false holddurationnanos: 0 - waiters: 0 + waiters: 1 waitingreaders: 0 - waitingwriters: 0 + waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -2913,7 +2966,7 @@ topklocksbyholdduration: maxwaitdurationnanos: 0 topklocksbywaitduration: - key: - - 100 + - 99 held: false holddurationnanos: 0 waiters: 1 @@ -2921,12 +2974,13 @@ topklocksbywaitduration: waitingwriters: 1 waitdurationnanos: 0 maxwaitdurationnanos: 0 -- key: [] +- key: + - 100 held: false holddurationnanos: 0 - waiters: 0 + waiters: 2 waitingreaders: 0 - waitingwriters: 0 + waitingwriters: 2 waitdurationnanos: 0 maxwaitdurationnanos: 0 - key: [] @@ -2942,7 +2996,8 @@ acquire r=req23 k=d durability=u ---- num=2 lock: "c" - res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 22, txn: 00000000-0000-0000-0000-000000000002 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: 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 373e37b74c23..026f7433a5bb 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 @@ -215,7 +215,8 @@ num=5 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] queued writers: @@ -239,15 +240,20 @@ print ---- num=5 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "d" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req1 ---- @@ -343,9 +349,11 @@ print ---- num=2 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req3 ---- @@ -441,8 +449,8 @@ release txn=txn2 span=a ---- num=2 lock: "a" - res: req: 5, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 active: true req: 6, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 6 lock: "b" @@ -464,18 +472,20 @@ print ---- num=2 lock: "a" - res: req: 5, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 active: true req: 6, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 6 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req5 ---- num=1 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req6 ---- @@ -564,9 +574,11 @@ print ---- num=2 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - res: req: 7, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req7 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable index 5a6ec45c2fc3..586c63fb5687 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable @@ -89,7 +89,8 @@ release txn=txn2 span=a ---- num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000001 guard-state r=req2 ---- @@ -99,7 +100,8 @@ acquire r=req2 k=c durability=u ---- num=2 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [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 40d0f7b1a05f..3385daf633d4 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 @@ -52,7 +52,8 @@ print ---- num=1 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 scan r=req1 ---- @@ -66,21 +67,24 @@ add-discovered r=req1 k=b txn=txn3 consult-finalized-txn-cache=true ---- num=1 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 # Txn is finalized and finalizedTxnCache is consulted. add-discovered r=req1 k=c txn=txn3 consult-finalized-txn-cache=true ---- num=1 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 # Txn is not finalized and finalizedTxnCache is consulted. add-discovered r=req1 k=d txn=txn4 consult-finalized-txn-cache=true ---- num=2 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000004, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access index bf953752db14..fa385b8882fc 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access @@ -60,7 +60,8 @@ release txn=txn1 span=a ---- num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 guard-state r=req2 ---- @@ -165,7 +166,8 @@ num=3 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -182,7 +184,8 @@ num=3 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -195,9 +198,11 @@ release txn=txn1 span=a ---- num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -212,10 +217,11 @@ print ---- num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -229,13 +235,15 @@ release txn=txn1 span=c ---- num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req5 ---- @@ -249,22 +257,26 @@ print ---- num=3 lock: "a" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req4 ---- num=2 lock: "b" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req5 ---- @@ -386,7 +398,8 @@ num=3 active: true req: 7, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" - res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 9, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -403,7 +416,8 @@ num=3 active: true req: 7, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "b" - res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 9, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -416,9 +430,11 @@ release txn=txn1 span=a ---- num=3 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 9, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -433,10 +449,11 @@ print ---- num=3 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000002 active: false req: 9, txn: 00000000-0000-0000-0000-000000000002 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -452,7 +469,8 @@ acquire r=req7 k=b durability=u ---- num=3 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" @@ -466,7 +484,8 @@ release txn=txn1 span=c ---- num=2 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -478,7 +497,8 @@ print ---- num=2 lock: "a" - res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -586,16 +606,19 @@ num=2 active: true req: 11, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 lock: "b" - res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 12, txn: 00000000-0000-0000-0000-000000000003 # req11 reserves "a" release txn=txn1 span=a ---- num=2 lock: "a" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 11, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 12, txn: 00000000-0000-0000-0000-000000000003 # req11 breaks the reservation at "b" guard-state r=req11 @@ -606,10 +629,11 @@ print ---- num=2 lock: "a" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 11, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 11, txn: 00000000-0000-0000-0000-000000000002 active: false req: 12, txn: 00000000-0000-0000-0000-000000000003 scan r=req11 @@ -620,7 +644,8 @@ acquire r=req11 k=b durability=u ---- num=2 lock: "a" - res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 11, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: 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 8a4921a2b126..20a0b1114779 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter @@ -119,7 +119,8 @@ release txn=txn2 span=a ---- num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" @@ -137,7 +138,8 @@ print ---- num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" @@ -153,12 +155,13 @@ release txn=txn2 span=c ---- num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 guard-state r=req1 @@ -173,15 +176,16 @@ print ---- num=3 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] waiting readers: req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 # req1 is done waiting. @@ -190,10 +194,11 @@ release txn=txn2 span=b ---- num=2 lock: "a" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 guard-state r=req1 @@ -204,7 +209,8 @@ dequeue r=req1 ---- num=1 lock: "c" - res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 2, 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 f6afc1ee3f73..4ffca5092bbc 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write @@ -112,14 +112,16 @@ release txn=txn1 span=a,d ---- num=3 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, txn: none distinguished req: 4 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req2 ---- @@ -150,15 +152,17 @@ print ---- num=3 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, txn: none active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 4 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 # Release the reservation at a. The first waiter is non-transactional so it will not acquire the # reservation. The second waiter will acquire the reservation. The non-transactional request will @@ -168,11 +172,14 @@ dequeue r=req2 ---- num=3 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000003 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req4 ---- @@ -186,14 +193,16 @@ print ---- num=3 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: none distinguished req: 4 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 # Release the reservation at b. The non-transactional waiter will be done at b, and when it gets # to c it will see a reservation holder with a higher sequence num and ignore it. @@ -202,9 +211,11 @@ dequeue r=req3 ---- num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req4 ---- @@ -218,9 +229,11 @@ print ---- num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 # Non-transactional request scans again and proceeds to evaluation and discovers a lock at c @@ -232,7 +245,8 @@ add-discovered r=req4 k=c txn=txn2 ---- num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -261,9 +275,11 @@ release txn=txn2 span=c ---- num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 guard-state r=req4 ---- @@ -279,15 +295,18 @@ dequeue r=req4 ---- num=2 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 dequeue r=req5 ---- num=1 lock: "a" - res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 + queued writers: + active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 dequeue r=req6 ---- 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 7a105f7fd556..3703767d349a 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded @@ -125,8 +125,8 @@ release txn=txn1 span=a ---- num=3 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -145,8 +145,8 @@ print ---- num=3 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] @@ -192,8 +192,8 @@ add-discovered r=req7 k=d txn=txn1 ---- num=4 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000002 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked index 36efae917e4f..8258d4d2e711 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked @@ -114,7 +114,8 @@ num=4 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 # --------------------------------------------------------------------------------- # req4 will scan the lock table with a Skip wait policy. It will not need to wait. @@ -191,7 +192,8 @@ num=4 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 # --------------------------------------------------------------------------------- # req5 is the same as req4, except is has a timestamp equal to txn1's to @@ -243,4 +245,5 @@ num=4 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" - res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/update b/pkg/kv/kvserver/concurrency/testdata/lock_table/update index 9415c5581992..bbe9ddb3d019 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/update +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/update @@ -339,8 +339,8 @@ update txn=txn1 ts=19,1 epoch=1 span=a ---- num=1 lock: "a" - res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 14.000000000,1, seq: 0 queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, 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 9298e10dfe42..c549fc70e547 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self @@ -84,8 +84,8 @@ release txn=txn1 span=a ---- num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 @@ -106,8 +106,8 @@ print ---- num=1 lock: "a" - res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: + active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index e1c08555ffc2..c7a265221335 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -876,14 +876,22 @@ func MVCCBlindPutProto( // table (i.e. unreplicated locks) and locks only stored in the persistent lock // table keyspace (i.e. replicated locks that have yet to be "discovered"). type LockTableView interface { - // IsKeyLockedByConflictingTxn returns whether the specified key is locked or - // reserved (see lockTable "reservations") by a conflicting transaction, given - // the caller's own desired locking strength. If so, true is returned. If the - // key is locked, the lock holder is also returned. Otherwise, if the key is - // reserved, nil is also returned. A transaction's own lock or reservation - // does not appear to be locked to itself (false is returned). The method is - // used by requests in conjunction with the SkipLocked wait policy to - // determine which keys they should skip over during evaluation. + // IsKeyLockedByConflictingTxn returns whether the specified key is claimed + // (see claimantTxn()) by a conflicting transaction in the lockTableGuard's + // snapshot of the lock table, given the caller's own desired locking + // strength. If so, true is returned. If the key is locked, the lock holder is + // also returned. Otherwise, if the key was claimed by a concurrent request + // still sequencing through the lock table, but the lock isn't held (yet), nil + // is also returned. + // + // If the lock has been claimed (held or otherwise) by the transaction itself, + // there's no conflict to speak of, so false is returned. In cases where the + // lock isn't held, but the lock has been claimed by the transaction itself, + // we do not make a distinction about which request claimed the key -- it + // could either be the request itself, or a different concurrent request from + // the same transaction; The specifics do not affect the caller. + // This method is used by requests in conjunction with the SkipLocked wait + // policy to determine which keys they should skip over during evaluation. IsKeyLockedByConflictingTxn(roachpb.Key, lock.Strength) (bool, *enginepb.TxnMeta) }