From 9146295b863f43da80401e82ffc2f4eaeba958fb Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Tue, 15 Dec 2020 09:06:21 -0500 Subject: [PATCH] concurrency: move finalizedTxnCache into lock table This is a cleanup in preparation for the future, and also has some, possibly minor, immediate benefits. In the future, the lock table will support multiple intents for the same key if all but one are known to be finalized. So the finalizedTxnCache belongs in the lock table data-structure. Additionally, we will support intent resolution without holding latches, which has some implications on data-structure consistency: request evaluation will not be allowed to add discovered intents to the lock table since the discovery may be stale. This PR is not changing this discovery behavior since we need it for now (due to interleaved intents), but it moves us along the path towards the lock table data-structure not relying on external behavior for maintaining its in-memory "cache" of locks. Specifically, removing intents from the lock table when the intent is still present in the engine is not principled. We currently do this in two places: - for optimizing limited scans: a later PR will fix this properly by checking the lock table after request evaluation, as outlined in #49973. - using the finalizedTxnCache in the lockTableWaiterImpl: this use is changed in this PR. The code in the lock table also does removal of intents before resolution, but there is a TODO to fix that in the future. It should be easier to do this with the behavior contained in the lock table. The immediate benefits, which may not have any practical significance, are: - We no longer resolve unreplicated locks -- they are simply removed. - A replicated lock is removed from the lock table data-structure only when the requester has finished a scan and is in a position to do resolution. Earlier one could remove the lock but block on another lock, and not do intent resolution on the first lock. This would cause wasteful evaluation of other requests. Informs #41720 Release note: None --- .../concurrency/concurrency_control.go | 16 +- .../concurrency/concurrency_manager.go | 12 +- pkg/kv/kvserver/concurrency/lock_table.go | 297 ++++++- .../kvserver/concurrency/lock_table_test.go | 44 +- .../kvserver/concurrency/lock_table_waiter.go | 106 +-- .../concurrency/lock_table_waiter_test.go | 46 +- .../clear_abandoned_intents | 401 ++++++++- .../concurrency_manager/wait_policy_error | 11 +- .../lock_table/clear_finalized_txn_locks | 829 ++++++++++++++++++ 9 files changed, 1595 insertions(+), 167 deletions(-) create mode 100644 pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index de7e90222cd8..acbf453f7492 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -567,6 +567,12 @@ type lockTable interface { // txn.WriteTimestamp. UpdateLocks(*roachpb.LockUpdate) error + // Informs the lock table that a transaction is finalized. This is used + // by the lock table in a best-effort manner to avoid waiting on locks + // of finalized transactions and telling the caller via + // lockTableGuard.ResolveBeforeEvaluation to resolve a batch of intents. + TransactionIsFinalized(*roachpb.Transaction) + // String returns a debug string representing the state of the lockTable. String() string } @@ -588,6 +594,11 @@ type lockTableGuard interface { // CurState returns the latest waiting state. CurState() waitingState + + // ResolveBeforeScanning lists the locks to resolve before scanning again. + // This must be called after the waiting state has transitioned to + // doneWaiting. + ResolveBeforeScanning() []roachpb.LockUpdate } // lockTableWaiter is concerned with waiting in lock wait-queues for locks held @@ -646,11 +657,6 @@ type lockTableWaiter interface { // and, in turn, remove this method. This will likely fall out of pulling // all replicated locks into the lockTable. WaitOnLock(context.Context, Request, *roachpb.Intent) *Error - - // ClearCaches wipes all caches maintained by the lockTableWaiter. This is - // primarily used to recover memory when a replica loses a lease. However, - // it is also used in tests to reset the state of the lockTableWaiter. - ClearCaches() } // txnWaitQueue holds a collection of wait-queues for transaction records. diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager.go b/pkg/kv/kvserver/concurrency/concurrency_manager.go index 2e5e53a8d5d4..f6218c7db179 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager.go @@ -72,6 +72,9 @@ func (c *Config) initDefaults() { func NewManager(cfg Config) Manager { cfg.initDefaults() m := new(managerImpl) + lt := &lockTableImpl{ + maxLocks: cfg.MaxLockTableSize, + } *m = managerImpl{ // TODO(nvanbenschoten): move pkg/storage/spanlatch to a new // pkg/storage/concurrency/latch package. Make it implement the @@ -82,14 +85,12 @@ func NewManager(cfg Config) Manager { cfg.SlowLatchGauge, ), }, - lt: &lockTableImpl{ - maxLocks: cfg.MaxLockTableSize, - }, + lt: lt, ltw: &lockTableWaiterImpl{ st: cfg.Settings, stopper: cfg.Stopper, ir: cfg.IntentResolver, - lm: m, + lt: lt, disableTxnPushing: cfg.DisableTxnPushing, }, // TODO(nvanbenschoten): move pkg/storage/txnwait to a new @@ -344,9 +345,6 @@ func (m *managerImpl) OnRangeLeaseUpdated(seq roachpb.LeaseSequence, isLeasehold const disable = true m.lt.Clear(disable) m.twq.Clear(disable) - // Also clear caches, since they won't be needed any time soon and - // consume memory. - m.ltw.ClearCaches() } } diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 554268f83d53..e587234ba404 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -185,6 +185,17 @@ type lockTableImpl struct { locks [spanset.NumSpanScope]treeMu maxLocks int64 + + // finalizedTxnCache is a small LRU cache that tracks transactions that + // were pushed and found to be finalized (COMMITTED or ABORTED). It is + // used as an optimization to avoid repeatedly pushing the transaction + // record when cleaning up the intents of an abandoned transaction. + // + // NOTE: it probably makes sense to maintain a single finalizedTxnCache + // across all Ranges on a Store instead of an individual cache per + // Range. For now, we don't do this because we don't share any state + // between separate concurrency.Manager instances. + finalizedTxnCache txnCache } var _ lockTable = &lockTableImpl{} @@ -256,6 +267,7 @@ var _ lockTable = &lockTableImpl{} // lockTableGuard that returns false from StartWaiting()). type lockTableGuardImpl struct { seqNum uint64 + lt *lockTableImpl // Information about this request. txn *enginepb.TxnMeta @@ -332,6 +344,10 @@ type lockTableGuardImpl struct { // (proportional to number of waiters). mustFindNextLockAfter bool } + // Locks to resolve before scanning again. Doesn't need to be protected by + // mu since should only be read after the caller has already synced with mu + // in realizing that it is doneWaiting. + toResolve []roachpb.LockUpdate } var _ lockTableGuard = &lockTableGuardImpl{} @@ -379,6 +395,10 @@ func (g *lockTableGuardImpl) ShouldWait() bool { return g.mu.startWait } +func (g *lockTableGuardImpl) ResolveBeforeScanning() []roachpb.LockUpdate { + return g.toResolve +} + func (g *lockTableGuardImpl) NewStateChan() chan struct{} { g.mu.Lock() defer g.mu.Unlock() @@ -431,7 +451,8 @@ func (g *lockTableGuardImpl) isSameTxnAsReservation(ws waitingState) bool { // Finds the next lock, after the current one, to actively wait at. If it // finds the next lock the request starts actively waiting there, else it is -// told that it is done waiting. +// told that it is done waiting. lockTableImpl.finalizedTxnCache is used to +// accumulate intents to resolve. // Acquires g.mu. func (g *lockTableGuardImpl) findNextLockAfter(notify bool) { spans := g.spans.GetSpans(g.sa, g.ss) @@ -443,6 +464,18 @@ func (g *lockTableGuardImpl) findNextLockAfter(notify bool) { span = &spans[g.index] resumingInSameSpan = true } + // Locks that transition to free because of the finalizedTxnCache are GC'd + // before returning. Note that these are only unreplicated locks. Replicated + // locks are handled via the g.toResolve. + var locksToGC [spanset.NumSpanScope][]*lockState + defer func() { + for i := 0; i < len(locksToGC); i++ { + if len(locksToGC[i]) > 0 { + g.lt.tryGCLocks(&g.lt.locks[i], locksToGC[i]) + } + } + }() + for span != nil { startKey := span.Key if resumingInSameSpan { @@ -468,26 +501,67 @@ func (g *lockTableGuardImpl) findNextLockAfter(notify bool) { // Else, past the lock where it stopped waiting. We may not // encounter that lock since it may have been garbage collected. } - if l.tryActiveWait(g, g.sa, notify) { + wait, transitionedToFree := l.tryActiveWait(g, g.sa, notify) + if transitionedToFree { + locksToGC[g.ss] = append(locksToGC[g.ss], l) + } + if wait { return } } resumingInSameSpan = false span = stepToNextSpan(g) } + if len(g.toResolve) > 0 { + j := 0 + // Some of the locks in g.toResolve may already have been claimed by + // another concurrent request and removed, or intent resolution could have + // happened while this request was waiting (after releasing latches). So + // we iterate over all the elements of toResolve and only keep the ones + // where removing the lock via the call to updateLockInternal is not a + // noop. + for i := range g.toResolve { + if heldByTxn := g.lt.updateLockInternal(&g.toResolve[i]); heldByTxn { + g.toResolve[j] = g.toResolve[i] + j++ + } + } + g.toResolve = g.toResolve[:j] + } g.mu.Lock() defer g.mu.Unlock() g.mu.state = waitingState{kind: doneWaiting} + // We are doneWaiting but may have some locks to resolve. There are + // two cases: + // - notify=false: the caller was already waiting and will look at this list + // of locks. + // - notify=true: this is a scan initiated by the caller, and it is holding + // latches. We need to tell it to "wait", so that it does this resolution + // first. startWait is currently false. This is the case handled below. if notify { + if len(g.toResolve) > 0 { + // Force caller to release latches and resolve intents. The first + // state it will see after releasing latches is doneWaiting, which + // will cause it to resolve intents. + g.mu.startWait = true + } g.notify() } } // Waiting writers in a lockState are wrapped in a queuedGuard. A waiting // writer is typically waiting in an active state, i.e., the -// lockTableGuardImpl.key refers to this lockState. However, breaking of -// reservations (see the comment on reservations below, in lockState) can -// cause a writer to be an inactive waiter. +// 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. +// - A discovered lock causes the discoverer to become an inactive waiter +// (until is 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. type queuedGuard struct { guard *lockTableGuardImpl active bool // protected by lockState.mu @@ -740,12 +814,12 @@ func (l *lockState) SetEndKey(v []byte) { l.endKey = v } // REQUIRES: l.mu is locked. func (l *lockState) String() string { var buf strings.Builder - l.Format(&buf) + l.Format(&buf, nil) return buf.String() } -// REQUIRES: l.mu is locked. -func (l *lockState) Format(buf *strings.Builder) { +// REQUIRES: l.mu is locked. finalizedTxnCache can be nil. +func (l *lockState) Format(buf *strings.Builder, finalizedTxnCache *txnCache) { fmt.Fprintf(buf, " lock: %s\n", l.key) if l.isEmptyLock() { fmt.Fprintln(buf, " empty") @@ -773,6 +847,19 @@ func (l *lockState) Format(buf *strings.Builder) { } else { fmt.Fprintf(b, "unrepl ") } + if finalizedTxnCache != nil { + finalizedTxn, ok := finalizedTxnCache.get(h.txn.ID) + if ok { + var statusStr string + switch finalizedTxn.Status { + case roachpb.COMMITTED: + statusStr = "committed" + case roachpb.ABORTED: + statusStr = "aborted" + } + fmt.Fprintf(b, "[holder finalized: %s] ", statusStr) + } + } fmt.Fprintf(b, "epoch: %d, seqs: [%d", h.txn.Epoch, h.seqs[0]) for j := 1; j < len(h.seqs); j++ { fmt.Fprintf(b, ", %d", h.seqs[j]) @@ -1028,33 +1115,104 @@ func (l *lockState) clearLockHolder() { // it is set to false when the call to tryActiveWait is happening due to an // event for a different request or transaction (like a lock release) since in // that case the channel is notified first and the call to tryActiveWait() -// happens later in lockTableGuard.CurState(). The return value is true iff -// it is actively waiting. +// happens later in lockTableGuard.CurState(). +// +// It uses the finalizedTxnCache to decide that the caller does not need to +// wait on a lock of a transaction that is already finalized. +// +// - For unreplicated locks, this method will silently remove the lock and +// proceed as normal. +// - For replicated locks the behavior is more complicated since we need to +// resolve the intent. We desire: +// A. batching of intent resolution. +// B. minimize races where intent resolution is being performed by multiple +// requests. +// C. minimize races where the intent has not yet been resolved but has been +// removed from the lock table, thereby causing some other request to +// evaluate wastefully and discover the intent. +// +// For A, the caller of tryActiveWait will accumulate the LockUpdates. For B, +// we only generate a LockUpdate here if this request is either a reader, or +// the first writer in the queue, i.e., it is only blocked by the lock +// holder. This prevents races between multiple writers in doing resolution +// but not between multiple readers and between readers and writers. We could +// be more conservative in only doing the intent resolution if the waiter was +// equivalent to a distinguished-waiter, but there it no guarantee that that +// distinguished waiter will do intent resolution in a timely manner (since +// it could block waiting on some other lock). Instead, the caller of +// tryActiveWait makes a best-effort to reduce racing (explained below). For +// C, the caller of tryActiveWait removes the lock from the in-memory +// data-structure only if the request does not need to wait anywhere, which +// means it will immediately proceed to intent resolution. Additionally, if +// the lock has already been removed, it suggests that some other request has +// already claimed intent resolution (or done it), so this request does not +// need to do the resolution. +// +// Ideally, we would strengthen B and C -- a request should make a claim on +// intent resolution for a set of keys, and will either resolve the intent, +// or due to an error will return that claim so others can do so. A +// replicated lock (intent) would not be removed from the in-memory +// data-structure until it was actually gone. +// TODO(sumeer): do this cleaner solution for batched intent resolution. +// +// In the future we'd like to augment the lockTable with an understanding of +// finalized but not yet resolved locks. These locks will allow conflicting +// transactions to proceed with evaluation without the need to first remove +// all traces of them via a round of replication. This is discussed in more +// detail in #41720. Specifically, see mention of "contention footprint" and +// COMMITTED_BUT_NOT_REMOVABLE. +// Also, resolving these locks/intents would proceed without latching, so we +// would not rely on MVCC scanning to add discovered locks to the lock table, +// since the discovered locks may be stale. +// +// The return value is true iff it is actively waiting. // Acquires l.mu, g.mu. -func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, notify bool) bool { +func (l *lockState) tryActiveWait( + g *lockTableGuardImpl, sa spanset.SpanAccess, notify bool, +) (wait bool, transitionedToFree bool) { l.mu.Lock() defer l.mu.Unlock() // It is possible that this lock is empty and has not yet been deleted. if l.isEmptyLock() { - return false + return false, false } // Lock is not empty. lockHolderTxn, lockHolderTS := l.getLockHolder() if lockHolderTxn != nil && g.isSameTxn(lockHolderTxn) { // Already locked by this txn. - return false + return false, false + } + + var replicatedLockFinalizedTxn *roachpb.Transaction + if lockHolderTxn != nil { + finalizedTxn, ok := g.lt.finalizedTxnCache.get(lockHolderTxn.ID) + if ok { + if l.holder.holder[lock.Replicated].txn == nil { + // Only held unreplicated. Release immediately. + l.clearLockHolder() + if l.lockIsFree() { + // Empty lock. + return false, true + } + lockHolderTxn = nil + // There is a reservation holder, which may be the caller itself, + // so fall through to the processing below. + } else { + replicatedLockFinalizedTxn = finalizedTxn + } + } } if sa == spanset.SpanReadOnly { if lockHolderTxn == nil { // Reads only care about locker, not a reservation. - return false + return false, false } // Locked by some other txn. if g.readTS.Less(lockHolderTS) { - return false + return false, false } g.mu.Lock() _, alsoHasStrongerAccess := g.mu.locks[l] @@ -1073,7 +1231,7 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, // timestamp that is not compatible with this request and it will wait // here -- there is no correctness issue with doing that. if alsoHasStrongerAccess { - return false + return false, false } } @@ -1084,7 +1242,7 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, } else { if l.reservation == g { // Already reserved by this request. - return false + 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 @@ -1093,7 +1251,7 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, 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. - return false + return false, false } waitForState.txn = l.reservation.txn } @@ -1109,18 +1267,18 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, // 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 + return false, false } - // Need to wait. - + // May need to wait. + wait = true g.mu.Lock() defer g.mu.Unlock() if sa == spanset.SpanReadWrite { + var qg *queuedGuard if _, inQueue := g.mu.locks[l]; inQueue { // Already in queue and must be in the right position, so mark as active // waiter there. We expect this to be rare. - var qg *queuedGuard for e := l.queuedWriters.Front(); e != nil; e = e.Next() { qqg := e.Value.(*queuedGuard) if qqg.guard == g { @@ -1131,10 +1289,12 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, if qg == nil { panic("lockTable bug") } + // Tentative. See below. qg.active = true } else { - // Not in queue so insert as active waiter. - qg := &queuedGuard{ + // Not in queue so insert as active waiter. The active waiter + // designation is tentative (see below). + qg = &queuedGuard{ guard: g, active: true, } @@ -1156,9 +1316,26 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, } g.mu.locks[l] = struct{}{} } + if replicatedLockFinalizedTxn != nil && l.queuedWriters.Front().Value.(*queuedGuard) == qg { + // First waiter, so should not wait. NB: this inactive waiter can be + // non-transactional. + qg.active = false + wait = false + } } else { - l.waitingReaders.PushFront(g) - g.mu.locks[l] = struct{}{} + if replicatedLockFinalizedTxn != nil { + // Don't add to waitingReaders since all readers in waitingReaders are + // active waiters, and this request is not an active waiter here. + wait = false + } else { + l.waitingReaders.PushFront(g) + g.mu.locks[l] = struct{}{} + } + } + if !wait { + g.toResolve = append( + g.toResolve, roachpb.MakeLockUpdate(replicatedLockFinalizedTxn, roachpb.Span{Key: l.key})) + return false, false } // Make it an active waiter. g.key = l.key @@ -1177,7 +1354,7 @@ func (l *lockState) tryActiveWait(g *lockTableGuardImpl, sa spanset.SpanAccess, if notify { g.notify() } - return true + return true, false } // Acquires this lock. Returns the list of guards that are done actively @@ -1363,6 +1540,7 @@ func (l *lockState) discoveredLock( case spanset.SpanReadWrite: // Immediately enter the lock's queuedWriters list. + // NB: this inactive waiter can be non-transactional. g.mu.Lock() _, presentHere := g.mu.locks[l] if !presentHere { @@ -1491,18 +1669,23 @@ func removeIgnored( // Tries to update the lock: noop if this lock is held by a different // transaction, else the lock is updated. Returns whether the lockState can be -// garbage collected. +// garbage collected, and whether it was held by the txn. // Acquires l.mu. -func (l *lockState) tryUpdateLock(up *roachpb.LockUpdate) (gc bool, err error) { +func (l *lockState) tryUpdateLock(up *roachpb.LockUpdate) (heldByTxn, gc bool) { l.mu.Lock() defer l.mu.Unlock() + if l.isEmptyLock() { + // Already free. This can happen when an unreplicated lock is removed in + // tryActiveWait due to the txn being in the finalizedTxnCache. + return false, true + } if !l.isLockedBy(up.Txn.ID) { - return false, nil + return false, false } if up.Status.IsFinalized() { l.clearLockHolder() gc = l.lockIsFree() - return gc, nil + return true, gc } txn := &up.Txn @@ -1560,7 +1743,7 @@ func (l *lockState) tryUpdateLock(up *roachpb.LockUpdate) (gc bool, err error) { if !isLocked { l.clearLockHolder() gc = l.lockIsFree() - return gc, nil + return true, gc } if advancedTs { @@ -1569,7 +1752,7 @@ func (l *lockState) tryUpdateLock(up *roachpb.LockUpdate) (gc bool, err error) { // Else no change for waiters. This can happen due to a race between different // callers of UpdateLocks(). - return false, nil + return true, false } // The lock holder timestamp has increased. Some of the waiters may no longer @@ -1710,6 +1893,7 @@ func (l *lockState) lockIsFree() (gc bool) { } // All waiting readers don't need to wait here anymore. + // NB: all waiting readers are by definition active waiters. for e := l.waitingReaders.Front(); e != nil; { g := e.Value.(*lockTableGuardImpl) curr := e @@ -1730,10 +1914,16 @@ func (l *lockState) lockIsFree() (gc bool) { curr := e e = e.Next() l.queuedWriters.Remove(curr) - if g == l.distinguishedWaiter { - l.distinguishedWaiter = nil + if qg.active { + if g == l.distinguishedWaiter { + l.distinguishedWaiter = nil + } + g.doneWaitingAtLock(false, l) + } else { + g.mu.Lock() + delete(g.mu.locks, l) + g.mu.Unlock() } - g.doneWaitingAtLock(false, l) } else { break } @@ -1779,6 +1969,7 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa if guard == nil { g = newLockTableGuardImpl() g.seqNum = atomic.AddUint64(&t.seqNum, 1) + g.lt = t g.txn = req.txnMeta() g.spans = req.LockSpans g.readTS = req.readConflictTimestamp() @@ -1795,6 +1986,7 @@ func (t *lockTableImpl) ScanAndEnqueue(req Request, guard lockTableGuard) lockTa g.mu.startWait = false g.mu.mustFindNextLockAfter = false g.mu.Unlock() + g.toResolve = g.toResolve[:0] } for ss := spanset.SpanScope(0); ss < spanset.NumSpanScope; ss++ { for sa := spanset.SpanAccess(0); sa < spanset.NumSpanAccess; sa++ { @@ -2041,6 +2233,13 @@ func (t *lockTableImpl) tryGCLocks(tree *treeMu, locks []*lockState) { // UpdateLocks implements the lockTable interface. func (t *lockTableImpl) UpdateLocks(up *roachpb.LockUpdate) error { + _ = t.updateLockInternal(up) + return nil +} + +// updateLockInternal is where the work for UpdateLocks is done. It +// returns whether there was a lock held by this txn. +func (t *lockTableImpl) updateLockInternal(up *roachpb.LockUpdate) (heldByTxn bool) { // NOTE: there is no need to synchronize with enabledMu here. Update only // accesses locks already in the lockTable, but a disabled lockTable will be // empty. If the lock-table scan below races with a concurrent call to clear @@ -2052,14 +2251,11 @@ func (t *lockTableImpl) UpdateLocks(up *roachpb.LockUpdate) error { ss = spanset.SpanLocal } tree := &t.locks[ss] - var err error var locksToGC []*lockState + heldByTxn = false changeFunc := func(l *lockState) { - gc, err2 := l.tryUpdateLock(up) - if err2 != nil { - err = err2 - return - } + held, gc := l.tryUpdateLock(up) + heldByTxn = heldByTxn || held if gc { locksToGC = append(locksToGC, l) } @@ -2079,7 +2275,7 @@ func (t *lockTableImpl) UpdateLocks(up *roachpb.LockUpdate) error { if len(locksToGC) > 0 { t.tryGCLocks(tree, locksToGC) } - return err + return heldByTxn } // Iteration helper for findNextLockAfter. Returns the next span to search @@ -2102,6 +2298,16 @@ func stepToNextSpan(g *lockTableGuardImpl) *spanset.Span { return nil } +// TransactionIsFinalized implements the lockTable interface. +func (t *lockTableImpl) TransactionIsFinalized(txn *roachpb.Transaction) { + // TODO(sumeer): We don't take any action for requests that are already + // waiting on locks held by txn. They need to take some action, like + // pushing, and resume their scan, to notice the change to this txn. We + // could be more proactive if we knew which locks in lockTableImpl were held + // by txn. + t.finalizedTxnCache.add(txn) +} + // Enable implements the lockTable interface. func (t *lockTableImpl) Enable(seq roachpb.LeaseSequence) { // Avoid disrupting other requests if the lockTable is already enabled. @@ -2129,6 +2335,9 @@ func (t *lockTableImpl) Clear(disable bool) { t.enabled = false } t.tryClearLocks(true /* force */) + // Also clear the finalized txn cache, since it won't be needed any time + // soon and consumes memory. + t.finalizedTxnCache.clear() } // For tests. @@ -2143,7 +2352,7 @@ func (t *lockTableImpl) String() string { for iter.First(); iter.Valid(); iter.Next() { l := iter.Cur() l.mu.Lock() - l.Format(&buf) + l.Format(&buf, &t.finalizedTxnCache) l.mu.Unlock() } tree.mu.RUnlock() diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 9b7a9127deaa..f4bf65007fe6 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -82,6 +82,11 @@ update txn= ts=[,] epoch= span=[,] [ignored-seq Updates locks for the named transaction. +txn-finalized txn= status=committed|aborted +---- + + Informs the lock table that the named transaction is finalized. + add-discovered r= k= txn= [lease-seq=] ---- @@ -181,6 +186,29 @@ func TestLockTableBasic(t *testing.T) { } return "" + case "txn-finalized": + var txnName string + d.ScanArgs(t, "txn", &txnName) + txnMeta, ok := txnsByName[txnName] + if !ok { + return fmt.Sprintf("txn %s not found", txnName) + } + txn := &roachpb.Transaction{ + TxnMeta: *txnMeta, + } + var statusStr string + d.ScanArgs(t, "status", &statusStr) + switch statusStr { + case "committed": + txn.Status = roachpb.COMMITTED + case "aborted": + txn.Status = roachpb.ABORTED + default: + return fmt.Sprintf("unknown txn status %s", statusStr) + } + lt.TransactionIsFinalized(txn) + return "" + case "new-request": // Seqnums for requests are numbered from 1 by lockTableImpl and // lockTableImpl.String() does not know about request names. Assigning @@ -374,9 +402,11 @@ func TestLockTableBasic(t *testing.T) { d.Fatalf(t, "unknown guard: %s", reqName) } var str string + stateTransition := false select { case <-g.NewStateChan(): str = "new: " + stateTransition = true default: str = "old: " } @@ -392,7 +422,19 @@ func TestLockTableBasic(t *testing.T) { case waitSelf: return str + "state=waitSelf" case doneWaiting: - return str + "state=doneWaiting" + var toResolveStr string + if stateTransition { + if toResolve := g.ResolveBeforeScanning(); len(toResolve) > 0 { + var buf strings.Builder + fmt.Fprintf(&buf, "\nIntents to resolve:") + for i := range toResolve { + fmt.Fprintf(&buf, "\n key=%s txn=%s status=%s", toResolve[i].Key, + toResolve[i].Txn.ID.Short(), toResolve[i].Status) + } + toResolveStr = buf.String() + } + } + return str + "state=doneWaiting" + toResolveStr } id := state.txn.ID var txnS string diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter.go b/pkg/kv/kvserver/concurrency/lock_table_waiter.go index c0758b87bfb7..c8d603b942e4 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter.go @@ -94,18 +94,7 @@ type lockTableWaiterImpl struct { st *cluster.Settings stopper *stop.Stopper ir IntentResolver - lm LockManager - - // finalizedTxnCache is a small LRU cache that tracks transactions that - // were pushed and found to be finalized (COMMITTED or ABORTED). It is - // used as an optimization to avoid repeatedly pushing the transaction - // record when cleaning up the intents of an abandoned transaction. - // - // NOTE: it probably makes sense to maintain a single finalizedTxnCache - // across all Ranges on a Store instead of an individual cache per - // Range. For now, we don't do this because we don't share any state - // between separate concurrency.Manager instances. - finalizedTxnCache txnCache + lt lockTable // When set, WriteIntentError are propagated instead of pushing // conflicting transactions. @@ -142,16 +131,6 @@ func (w *lockTableWaiterImpl) WaitOn( var timer *timeutil.Timer var timerC <-chan time.Time var timerWaitingState waitingState - // Used to defer the resolution of duplicate intents. Intended to allow - // batching of intent resolution while cleaning up after abandoned txns. A - // request may begin deferring intent resolution and then be forced to wait - // again on other locks. This is ok, as the request that deferred intent - // resolution will often be the new reservation holder for those intents' - // keys. Even when this is not the case (e.g. the request is read-only so it - // can't hold reservations), any other requests that slip ahead will simply - // re-discover the intent(s) during evaluation and resolve them themselves. - var deferredResolution []roachpb.LockUpdate - defer w.resolveDeferredIntents(ctx, &err, &deferredResolution) for { select { case <-newStateC: @@ -219,54 +198,6 @@ func (w *lockTableWaiterImpl) WaitOn( continue } - // If we know that a lock holder is already finalized (COMMITTED - // or ABORTED), there's no reason to push it again. Instead, we - // can skip directly to intent resolution. - // - // As an optimization, we defer the intent resolution until the - // we're done waiting on all conflicting locks in this function. - // This allows us to accumulate a group of intents to resolve - // and send them together as a batch. - // - // Remember that if the lock is held, there will be at least one - // waiter with livenessPush = true (the distinguished waiter), - // so at least one request will enter this branch and perform - // the cleanup on behalf of all other waiters. - if livenessPush { - if pusheeTxn, ok := w.finalizedTxnCache.get(state.txn.ID); ok { - resolve := roachpb.MakeLockUpdate(pusheeTxn, roachpb.Span{Key: state.key}) - deferredResolution = append(deferredResolution, resolve) - - // Inform the LockManager that the lock has been updated with a - // finalized status so that it gets removed from the lockTable - // and we are allowed to proceed. - // - // For unreplicated locks, this is all that is needed - the - // lockTable is the source of truth so, once removed, the - // unreplicated lock is gone. It is perfectly valid for us to - // instruct the lock to be released because we know that the - // lock's owner is finalized. - // - // For replicated locks, this is a bit of a lie. The lock hasn't - // actually been updated yet, but we will be conducting intent - // resolution in the future (before we observe the corresponding - // MVCC state). This is safe because we already handle cases - // where locks exist only in the MVCC keyspace and not in the - // lockTable. - // - // In the future, we'd like to make this more explicit. - // Specifically, we'd like to augment the lockTable with an - // understanding of finalized but not yet resolved locks. These - // locks will allow conflicting transactions to proceed with - // evaluation without the need to first remove all traces of - // them via a round of replication. This is discussed in more - // detail in #41720. Specifically, see mention of "contention - // footprint" and COMMITTED_BUT_NOT_REMOVABLE. - w.lm.OnLockUpdated(ctx, &deferredResolution[len(deferredResolution)-1]) - continue - } - } - // The request should push to detect abandoned locks due to // failed transaction coordinators, detect deadlocks between // transactions, or both, but only after delay. This delay @@ -334,7 +265,19 @@ func (w *lockTableWaiterImpl) WaitOn( // waiting, re-acquire latches, and check the lockTable again for // any new conflicts. If it find none, it can proceed with // evaluation. - return nil + // Note that the lockTable "claims" the list to resolve when this + // waiter is transitioning to doneWaiting, to increase the likelihood + // that this waiter will indeed do the resolution. However, it is + // possible for this transition to doneWaiting to race with + // cancellation of the request and slip in after the cancellation and + // before lockTable.Dequeue() is called. This will result in the locks + // being removed from the lockTable data-structure without subsequent + // resolution. Another requester will discover these locks during + // evaluation and add them back to the lock table data-structure. See + // the comment in lockTableImpl.tryActiveWait for the proper way to + // remove this and other evaluation races. + toResolve := guard.ResolveBeforeScanning() + return w.resolveDeferredIntents(ctx, toResolve) default: panic("unexpected waiting state") @@ -415,11 +358,6 @@ func (w *lockTableWaiterImpl) WaitOnLock( }) } -// ClearCaches implements the lockTableWaiter interface. -func (w *lockTableWaiterImpl) ClearCaches() { - w.finalizedTxnCache.clear() -} - // pushLockTxn pushes the holder of the provided lock. // // The method blocks until the lock holder transaction experiences a state @@ -482,7 +420,7 @@ func (w *lockTableWaiterImpl) pushLockTxn( // avoids needing to push it again if we find another one of its locks and // allows for batching of intent resolution. if pusheeTxn.Status.IsFinalized() { - w.finalizedTxnCache.add(pusheeTxn) + w.lt.TransactionIsFinalized(pusheeTxn) } // If the push succeeded then the lock holder transaction must have @@ -501,6 +439,10 @@ func (w *lockTableWaiterImpl) pushLockTxn( // would not be useful until we begin eagerly updating a transaction's // record upon rollbacks to savepoints. // + // TODO(sumeer): it is possible that the lock is an unreplicated lock, + // for which doing intent resolution is unnecessary -- we only need + // to remove it from the lock table data-structure. + // // Update the conflicting lock to trigger the desired state transition in // the lockTable itself, which will allow the request to proceed. // @@ -621,14 +563,14 @@ func (w *lockTableWaiterImpl) pushHeader(req Request) roachpb.Header { // nil. The batch of intents may be resolved more efficiently than if they were // resolved individually. func (w *lockTableWaiterImpl) resolveDeferredIntents( - ctx context.Context, err **Error, deferredResolution *[]roachpb.LockUpdate, -) { - if (*err != nil) || (len(*deferredResolution) == 0) { - return + ctx context.Context, deferredResolution []roachpb.LockUpdate, +) *Error { + if len(deferredResolution) == 0 { + return nil } // See pushLockTxn for an explanation of these options. opts := intentresolver.ResolveOptions{Poison: true} - *err = w.ir.ResolveIntents(ctx, *deferredResolution, opts) + return w.ir.ResolveIntents(ctx, deferredResolution, opts) } // watchForNotifications selects on the provided channel and watches for any diff --git a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go index 0f1f44deb0fc..2ae9ee912137 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_waiter_test.go @@ -60,6 +60,7 @@ type mockLockTableGuard struct { state waitingState signal chan struct{} stateObserved chan struct{} + toResolve []roachpb.LockUpdate } // mockLockTableGuard implements the lockTableGuard interface. @@ -72,17 +73,20 @@ func (g *mockLockTableGuard) CurState() waitingState { } return s } +func (g *mockLockTableGuard) ResolveBeforeScanning() []roachpb.LockUpdate { + return g.toResolve +} func (g *mockLockTableGuard) notify() { g.signal <- struct{}{} } -// mockLockTableGuard implements the LockManager interface. -func (g *mockLockTableGuard) OnLockAcquired(_ context.Context, _ *roachpb.LockAcquisition) { - panic("unimplemented") +// mockLockTable overrides TransactionIsFinalized, which is the only LockTable +// method that should be called in this test. +type mockLockTable struct { + lockTableImpl + txnFinalizedFn func(txn *roachpb.Transaction) } -func (g *mockLockTableGuard) OnLockUpdated(_ context.Context, up *roachpb.LockUpdate) { - if g.state.held && g.state.txn.ID == up.Txn.ID && g.state.key.Equal(up.Key) { - g.state = waitingState{kind: doneWaiting} - g.notify() - } + +func (lt *mockLockTable) TransactionIsFinalized(txn *roachpb.Transaction) { + lt.txnFinalizedFn(txn) } func setupLockTableWaiterTest() (*lockTableWaiterImpl, *mockIntentResolver, *mockLockTableGuard) { @@ -97,7 +101,7 @@ func setupLockTableWaiterTest() (*lockTableWaiterImpl, *mockIntentResolver, *moc st: st, stopper: stop.NewStopper(), ir: ir, - lm: guard, + lt: &mockLockTable{}, } return w, ir, guard } @@ -306,9 +310,13 @@ func testWaitPush(t *testing.T, k waitKind, makeReq func() Request, expPushTS hl resp := &roachpb.Transaction{TxnMeta: *pusheeArg, Status: roachpb.ABORTED} // If the lock is held, we'll try to resolve it now that - // we know the holder is ABORTED. Otherwide, immediately + // we know the holder is ABORTED. Otherwise, immediately // tell the request to stop waiting. if lockHeld { + w.lt.(*mockLockTable).txnFinalizedFn = func(txn *roachpb.Transaction) { + require.Equal(t, pusheeTxn.ID, txn.ID) + require.Equal(t, roachpb.ABORTED, txn.Status) + } ir.resolveIntent = func(_ context.Context, intent roachpb.LockUpdate) *Error { require.Equal(t, keyA, intent.Key) require.Equal(t, pusheeTxn.ID, intent.Txn.ID) @@ -452,6 +460,10 @@ func testErrorWaitPush(t *testing.T, k waitKind, makeReq func() Request, expPush // Next, we'll try to resolve the lock now that we know the // holder is ABORTED. + w.lt.(*mockLockTable).txnFinalizedFn = func(txn *roachpb.Transaction) { + require.Equal(t, pusheeTxn.ID, txn.ID) + require.Equal(t, roachpb.ABORTED, txn.Status) + } ir.resolveIntent = func(_ context.Context, intent roachpb.LockUpdate) *Error { require.Equal(t, keyA, intent.Key) require.Equal(t, pusheeTxn.ID, intent.Txn.ID) @@ -544,19 +556,17 @@ func TestLockTableWaiterDeferredIntentResolverError(t *testing.T) { } keyA := roachpb.Key("keyA") pusheeTxn := makeTxnProto("pushee") - - // Add the conflicting txn to the finalizedTxnCache so that the request - // avoids the transaction record push and defers the intent resolution. + // Make the pusheeTxn ABORTED so that the request avoids the transaction + // record push and defers the intent resolution. pusheeTxn.Status = roachpb.ABORTED - w.finalizedTxnCache.add(&pusheeTxn) g.state = waitingState{ - kind: waitForDistinguished, - txn: &pusheeTxn.TxnMeta, - key: keyA, - held: true, + kind: doneWaiting, guardAccess: spanset.SpanReadWrite, } + g.toResolve = []roachpb.LockUpdate{ + roachpb.MakeLockUpdate(&pusheeTxn, roachpb.Span{Key: keyA}), + } g.notify() // Errors are propagated when observed while resolving batches of intents. 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 cc08def4560d..ed110c41bfd4 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents @@ -193,7 +193,7 @@ global: num=2 lock: "a" res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 local: num=0 @@ -223,7 +223,7 @@ global: num=3 lock: "b" res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 local: num=0 @@ -257,3 +257,400 @@ finish req=req1 reset namespace ---- + +# --------------------------------------------------------------------------- +# A scan finds 2 intents and 2 unreplicated locks from the same txn. When the +# txn is abandoned, only the 2 intents need to be resolved, since it is +# sufficient to remove the unreplicated locks from the lock table. +# --------------------------------------------------------------------------- + +new-txn name=txn1 ts=10,1 epoch=0 +---- + +new-txn name=txn2 ts=10,1 epoch=0 +---- + +new-request name=req1 txn=txn1 ts=10,1 + scan key=a endkey=z +---- + +sequence req=req1 +---- +[1] sequence req1: sequencing request +[1] sequence req1: acquiring latches +[1] sequence req1: scanning lock table for conflicting locks +[1] sequence req1: sequencing complete, returned guard + +handle-write-intent-error req=req1 lease-seq=1 + intent txn=txn2 key=a + intent txn=txn2 key=b +---- +[2] handle write intent error req1: handled conflicting intents on "a", "b", released latches + +debug-lock-table +---- +global: num=2 + lock: "a" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +new-request name=req2 txn=txn2 ts=10,1 + put key=g value=v1 + put key=h value=v2 +---- + +sequence req=req2 +---- +[3] sequence req2: sequencing request +[3] sequence req2: acquiring latches +[3] sequence req2: scanning lock table for conflicting locks +[3] sequence req2: sequencing complete, returned guard + +on-lock-acquired req=req2 key=g dur=u +---- +[-] acquire lock: txn 00000002 @ g + +on-lock-acquired req=req2 key=h dur=u +---- +[-] acquire lock: txn 00000002 @ h + +finish req=req2 +---- +[-] finish req2: finishing request + +debug-lock-table +---- +global: num=4 + lock: "a" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "h" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +sequence req=req1 +---- +[4] sequence req1: re-sequencing request +[4] sequence req1: acquiring latches +[4] sequence req1: scanning lock table for conflicting locks +[4] sequence req1: waiting in lock wait-queues +[4] sequence req1: pushing timestamp of txn 00000002 above 10.000000000,1 +[4] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction + +debug-lock-table +---- +global: num=4 + lock: "a" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + waiting readers: + req: 3, txn: 00000001-0000-0000-0000-000000000000 + distinguished req: 3 + lock: "b" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "h" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] +local: num=0 + +on-txn-updated txn=txn2 status=aborted +---- +[-] update txn: aborting txn2 +[4] sequence req1: resolving intent "a" for txn 00000002 with ABORTED status +[4] sequence req1: resolving a batch of 1 intent(s) +[4] sequence req1: resolving intent "b" for txn 00000002 with ABORTED status +[4] sequence req1: acquiring latches +[4] sequence req1: scanning lock table for conflicting locks +[4] sequence req1: sequencing complete, returned guard + +debug-lock-table +---- +global: num=0 +local: num=0 + +finish req=req1 +---- +[-] finish req1: finishing request + +reset namespace +---- + +# --------------------------------------------------------------------------- +# req1 from txn1 wants to write c, d, e. req2 from txn2 wants to read a, b, d. +# All are held by other txns. When txn3 holding a, c, d is aborted, req1 resolves +# c, skips waiting at d, but does not remove the lock since it is not yet ready +# to resolve it, since it needs to wait at e. req2 resolves a and waits at b. +# When txn4, holding b, is aborted, req2 can resolve both b and d. +# --------------------------------------------------------------------------- + +new-txn name=txn1 ts=12,1 epoch=0 +---- + +new-txn name=txn2 ts=11,1 epoch=0 +---- + +new-txn name=txn3 ts=10,1 epoch=0 +---- + +new-txn name=txn4 ts=10,1 epoch=0 +---- + +new-txn name=txn5 ts=10,1 epoch=0 +---- + +new-request name=req1 txn=txn1 ts=12,1 + put key=c value=v1 + put key=d value=v1 + put key=e value=v1 +---- + +sequence req=req1 +---- +[1] sequence req1: sequencing request +[1] sequence req1: acquiring latches +[1] sequence req1: scanning lock table for conflicting locks +[1] sequence req1: sequencing complete, returned guard + +# Normally req1 will not discover write intents for c, d, e in one shot, since +# it consists of multiple requests, and we only discover intents a request at +# a time (though a single request can discover multiple intents), but we do +# this for shortening the test. +handle-write-intent-error req=req1 lease-seq=1 + intent txn=txn3 key=c + intent txn=txn3 key=d + intent txn=txn5 key=e +---- +[2] handle write intent error req1: handled conflicting intents on "c", "d", "e", released latches + +debug-lock-table +---- +global: num=3 + lock: "c" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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 epoch: 0, seqs: [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: + active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 +local: num=0 + +sequence req=req1 +---- +[3] sequence req1: re-sequencing request +[3] sequence req1: acquiring latches +[3] sequence req1: scanning lock table for conflicting locks +[3] sequence req1: waiting in lock wait-queues +[3] sequence req1: pushing txn 00000003 to abort +[3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction + +debug-lock-table +---- +global: num=3 + lock: "c" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 + distinguished req: 5 + lock: "d" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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: + active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 +local: num=0 + +new-request name=req3 txn=txn3 ts=10,1 + put key=a value=v3 +---- + +sequence req=req3 +---- +[4] sequence req3: sequencing request +[4] sequence req3: acquiring latches +[4] sequence req3: scanning lock table for conflicting locks +[4] sequence req3: sequencing complete, returned guard + +on-lock-acquired req=req3 key=a dur=u +---- +[-] acquire lock: txn 00000003 @ a + +finish req=req3 +---- +[-] finish req3: finishing request + +new-request name=req4 txn=txn4 ts=10,1 + put key=b value=v4 +---- + +sequence req=req4 +---- +[5] sequence req4: sequencing request +[5] sequence req4: acquiring latches +[5] sequence req4: scanning lock table for conflicting locks +[5] sequence req4: sequencing complete, returned guard + +on-lock-acquired req=req4 key=b dur=u +---- +[-] acquire lock: txn 00000004 @ b + +finish req=req4 +---- +[-] finish req4: finishing request + +debug-lock-table +---- +global: num=5 + lock: "a" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 + distinguished req: 5 + lock: "d" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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: + active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 +local: num=0 + +new-request name=req2 txn=txn2 ts=11,1 + scan key=a endkey=c + scan key=d endkey=e +---- + +sequence req=req2 +---- +[6] sequence req2: sequencing request +[6] sequence req2: acquiring latches +[6] sequence req2: scanning lock table for conflicting locks +[6] sequence req2: waiting in lock wait-queues +[6] sequence req2: pushing timestamp of txn 00000003 above 11.000000000,1 +[6] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction + +debug-lock-table +---- +global: num=5 + lock: "a" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + waiting readers: + req: 8, txn: 00000002-0000-0000-0000-000000000000 + distinguished req: 8 + lock: "b" + holder: txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 + distinguished req: 5 + lock: "d" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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: + active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 +local: num=0 + +# req1 resolves intent c, skips waiting at d, and waits at e. req2 resolves a, +# and waits at b. +on-txn-updated txn=txn3 status=aborted +---- +[-] update txn: aborting txn3 +[3] sequence req1: resolving intent "c" for txn 00000003 with ABORTED status +[3] sequence req1: pushing txn 00000005 to abort +[3] sequence req1: blocked on select in concurrency_test.(*cluster).PushTransaction +[6] sequence req2: resolving intent "a" for txn 00000003 with ABORTED status +[6] sequence req2: pushing timestamp of txn 00000004 above 11.000000000,1 +[6] sequence req2: blocked on select in concurrency_test.(*cluster).PushTransaction + +debug-lock-table +---- +global: num=4 + lock: "b" + holder: txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] + waiting readers: + 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 + lock: "d" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [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: + active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 + distinguished req: 5 +local: num=0 + +# The txn holding b is aborted. At this point req2 can resolve both b and d +# and continue to evaluation. +on-txn-updated txn=txn4 status=aborted +---- +[-] update txn: aborting txn4 +[6] sequence req2: resolving intent "b" for txn 00000004 with ABORTED status +[6] sequence req2: resolving a batch of 1 intent(s) +[6] sequence req2: resolving intent "d" for txn 00000003 with ABORTED status +[6] sequence req2: acquiring latches +[6] sequence req2: scanning lock table for conflicting locks +[6] sequence req2: sequencing complete, returned guard + +debug-lock-table +---- +global: num=3 + lock: "c" + res: req: 5, txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,1, seq: 0 + lock: "d" + res: req: 5, txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,1, seq: 0 + lock: "e" + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 + distinguished req: 5 +local: num=0 + +finish req=req2 +---- +[-] finish req2: finishing request + +# Tht txn holding e is aborted, so req1 can resolve e and proceed to +# evaluation. +on-txn-updated txn=txn5 status=aborted +---- +[-] update txn: aborting txn5 +[3] sequence req1: resolving intent "e" for txn 00000005 with ABORTED status +[3] sequence req1: acquiring latches +[3] sequence req1: scanning lock table for conflicting locks +[3] sequence req1: sequencing complete, returned guard + +finish req=req1 +---- +[-] finish req1: finishing request + +debug-lock-table +---- +global: num=0 +local: num=0 + +reset namespace +---- 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 1a174ac1ad1f..4c88fe2ad979 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error @@ -108,7 +108,7 @@ sequence req=reqNoWait1 # ------------------------------------------------------------- # Read-only request with WaitPolicy_Error hits abandoned lock. -# The request resolves the abandoned lock and proceeds. +# The request removes the abandoned unreplicated lock and proceeds. # ------------------------------------------------------------- on-txn-updated txn=txn1 status=committed @@ -123,11 +123,6 @@ sequence req=reqNoWait1 [5] sequence reqNoWait1: sequencing request [5] sequence reqNoWait1: acquiring latches [5] sequence reqNoWait1: scanning lock table for conflicting locks -[5] sequence reqNoWait1: waiting in lock wait-queues -[5] sequence reqNoWait1: pushing txn 00000001 to check if abandoned -[5] sequence reqNoWait1: resolving intent "k" for txn 00000001 with COMMITTED status -[5] sequence reqNoWait1: acquiring latches -[5] sequence reqNoWait1: scanning lock table for conflicting locks [5] sequence reqNoWait1: sequencing complete, returned guard finish req=reqNoWait1 @@ -244,7 +239,7 @@ sequence req=reqNoWait4 [12] sequence reqNoWait4: acquiring latches [12] sequence reqNoWait4: scanning lock table for conflicting locks [12] sequence reqNoWait4: waiting in lock wait-queues -[12] sequence reqNoWait4: pushing txn 00000002 to check if abandoned +[12] sequence reqNoWait4: resolving a batch of 1 intent(s) [12] sequence reqNoWait4: resolving intent "k5" for txn 00000002 with ABORTED status [12] sequence reqNoWait4: acquiring latches [12] sequence reqNoWait4: scanning lock table for conflicting locks @@ -262,7 +257,7 @@ debug-lock-table ---- global: num=1 lock: "k4" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] local: num=0 reset 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 new file mode 100644 index 000000000000..348e7bfd3879 --- /dev/null +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks @@ -0,0 +1,829 @@ +new-lock-table maxlocks=10000 +---- + +new-txn txn=txn1 ts=12,1 epoch=0 +---- + +new-txn txn=txn2 ts=10,1 epoch=0 +---- + +new-txn txn=txn3 ts=11,1 epoch=0 +---- + +new-txn txn=txn4 ts=11,1 epoch=0 +---- + +# ----------------------------------------------------------------------------- +# req1 waits for replicated locks held by txn2, txn3, and unreplicated lock +# held by txn4. When txn2 is finalized and req1 scans, it notices it no longer +# needs to wait for txn2 locks, but then waits on the lock held by txn4. +# Finalization of txn3 and explicit release of the lock held by txn3 causes +# req1 to finish scanning. It needs to resolve the locks held by txn2, txn3. +# ----------------------------------------------------------------------------- + +new-request r=req1 txn=txn1 ts=10,1 spans=w@a+w@b+w@c+w@d+w@e +---- + +new-request r=req2 txn=txn4 ts=11,1 spans=w@c +---- + +scan r=req1 +---- +start-waiting: false + +add-discovered r=req1 k=a txn=txn2 +---- +global: num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +add-discovered r=req1 k=b txn=txn2 +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +add-discovered r=req1 k=d txn=txn3 +---- +global: num=3 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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,1, info: repl epoch: 0, seqs: [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 epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +add-discovered r=req1 k=e txn=txn3 +---- +global: num=4 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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,1, info: repl epoch: 0, seqs: [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 epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "e" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +scan r=req2 +---- +start-waiting: false + +acquire r=req2 k=c durability=u +---- +global: num=5 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "e" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +dequeue r=req2 +---- +global: num=5 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [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,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "e" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +scan r=req1 +---- +start-waiting: true + +print +---- +global: num=5 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 + distinguished req: 1 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "e" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +txn-finalized txn=txn2 status=aborted +---- + +scan r=req1 +---- +start-waiting: true + +guard-state r=req1 +---- +new: state=waitForDistinguished txn=txn4 key="c" held=true guard-access=write + +print +---- +global: num=5 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + distinguished req: 1 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] + queued writers: + active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 + distinguished req: 1 + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "e" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +txn-finalized txn=txn3 status=committed +---- + +release txn=txn4 span=c +---- +global: num=5 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + distinguished req: 1 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [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 + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 + lock: "e" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] + queued writers: + active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + + +guard-state r=req1 +---- +new: state=doneWaiting +Intents to resolve: + key="a" txn=00000000 status=ABORTED + key="b" txn=00000000 status=ABORTED + key="d" txn=00000000 status=COMMITTED + key="e" txn=00000000 status=COMMITTED + +print +---- +global: num=5 + lock: "a" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + lock: "b" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + lock: "c" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + lock: "d" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + lock: "e" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 +local: num=0 + +dequeue r=req1 +---- +global: num=0 +local: num=0 + +clear +---- +global: num=0 +local: num=0 + +# ----------------------------------------------------------------------------- +# req3 waits for replicated and unreplicated locks held by txn2. When txn2 is +# finalized, the unreplicated lock is removed and the replicated locks are in +# the list of locks to resolve. +# ----------------------------------------------------------------------------- + +new-request r=req3 txn=txn1 ts=10,1 spans=w@a+w@b+w@c +---- + +new-request r=req4 txn=txn2 ts=11,1 spans=w@b +---- + +scan r=req3 +---- +start-waiting: false + +add-discovered r=req3 k=a txn=txn2 +---- +global: num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +add-discovered r=req3 k=c txn=txn2 +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +scan r=req4 +---- +start-waiting: false + +acquire r=req4 k=b durability=u +---- +global: num=3 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +dequeue r=req4 +---- +global: num=3 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +txn-finalized txn=txn2 status=aborted +---- + +scan r=req3 +---- +start-waiting: true + +guard-state r=req3 +---- +new: state=doneWaiting +Intents to resolve: + key="a" txn=00000000 status=ABORTED + key="c" txn=00000000 status=ABORTED + +print +---- +global: num=2 + lock: "a" + res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 + lock: "c" + res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 +local: num=0 + +dequeue r=req3 +---- +global: num=0 +local: num=0 + +clear +---- +global: num=0 +local: num=0 + +# ----------------------------------------------------------------------------- +# req5 waits for replicated locks held by txn2. When txn2 is +# finalized, it is first noticed by req6, but it has to wait behind req5. Eventually, +# req5 notices the finalization (via pushing) and scans again and resolves. +# ----------------------------------------------------------------------------- + +new-request r=req5 txn=txn1 ts=12,1 spans=w@a+w@b +---- + +new-request r=req6 txn=txn3 ts=12,1 spans=w@a +---- + +scan r=req5 +---- +start-waiting: false + +add-discovered r=req5 k=a txn=txn2 +---- +global: num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +add-discovered r=req5 k=b txn=txn2 +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +scan r=req5 +---- +start-waiting: true + +print +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: true req: 5, txn: 00000000-0000-0000-0000-000000000001 + distinguished req: 5 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +txn-finalized txn=txn2 status=aborted +---- + +scan r=req6 +---- +start-waiting: true + +guard-state r=req6 +---- +new: state=waitFor txn=txn2 key="a" held=true guard-access=write + +guard-state r=req5 +---- +new: state=waitForDistinguished txn=txn2 key="a" held=true guard-access=write + +print +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + queued writers: + active: true req: 5, txn: 00000000-0000-0000-0000-000000000001 + active: true req: 6, txn: 00000000-0000-0000-0000-000000000003 + distinguished req: 5 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +release txn=txn2 span=a +---- +global: num=2 + lock: "a" + res: req: 5, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 + queued writers: + active: true req: 6, txn: 00000000-0000-0000-0000-000000000003 + distinguished req: 6 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + queued writers: + active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +guard-state r=req6 +---- +new: state=waitForDistinguished txn=txn1 key="a" held=false guard-access=write + +guard-state r=req5 +---- +new: state=doneWaiting +Intents to resolve: + key="b" txn=00000000 status=ABORTED + +print +---- +global: num=2 + lock: "a" + res: req: 5, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 + queued writers: + 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 +local: num=0 + +dequeue r=req5 +---- +global: num=1 + lock: "a" + res: req: 6, txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, seq: 0 +local: num=0 + +dequeue r=req6 +---- +global: num=0 +local: num=0 + +clear +---- +global: num=0 +local: num=0 + +# ----------------------------------------------------------------------------- +# req7 waits for replicated locks held by txn2. When txn2 is finalized, it is +# first noticed by req8, which does not need to wait for req7 since it is a +# reader. +# ----------------------------------------------------------------------------- + +new-request r=req7 txn=txn1 ts=12,1 spans=w@a+w@b +---- + +new-request r=req8 txn=txn3 ts=12,1 spans=r@a+r@b +---- + +scan r=req7 +---- +start-waiting: false + +add-discovered r=req7 k=a txn=txn2 +---- +global: num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +add-discovered r=req7 k=b txn=txn2 +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +scan r=req7 +---- +start-waiting: true + +guard-state r=req7 +---- +new: state=waitForDistinguished txn=txn2 key="a" held=true guard-access=write + +print +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: true req: 7, txn: 00000000-0000-0000-0000-000000000001 + distinguished req: 7 + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 +local: num=0 + +txn-finalized txn=txn2 status=aborted +---- + +scan r=req8 +---- +start-waiting: true + +guard-state r=req8 +---- +new: state=doneWaiting +Intents to resolve: + key="a" txn=00000000 status=ABORTED + key="b" txn=00000000 status=ABORTED + +guard-state r=req7 +---- +new: state=doneWaiting + +print +---- +global: num=2 + lock: "a" + res: req: 7, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 + lock: "b" + res: req: 7, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 +local: num=0 + +dequeue r=req7 +---- +global: num=0 +local: num=0 + +dequeue r=req8 +---- +global: num=0 +local: num=0 + +clear +---- +global: num=0 +local: num=0 + +# ----------------------------------------------------------------------------- +# Replicated locks are held by txn3, txn4. When txn3 is finalized, req9 notices +# it but is unable to resolve since it has to wait for txn4. req10 comes through +# and resolves txn3 locks before req9. +# ----------------------------------------------------------------------------- + +new-request r=req9 txn=txn1 ts=12,1 spans=r@a+r@b+r@c+r@d +---- + +new-request r=req10 txn=txn2 ts=12,1 spans=r@a+r@b +---- + +scan r=req9 +---- +start-waiting: false + +add-discovered r=req9 k=a txn=txn3 +---- +global: num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +add-discovered r=req9 k=b txn=txn3 +---- +global: num=2 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +add-discovered r=req9 k=c txn=txn4 +---- +global: num=3 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +add-discovered r=req9 k=d txn=txn4 +---- +global: num=4 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +txn-finalized txn=txn3 status=aborted +---- + +scan r=req9 +---- +start-waiting: true + +guard-state r=req9 +---- +new: state=waitForDistinguished txn=txn4 key="c" held=true guard-access=read + +print +---- +global: num=4 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + waiting readers: + req: 9, txn: 00000000-0000-0000-0000-000000000001 + distinguished req: 9 + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +scan r=req10 +---- +start-waiting: true + +guard-state r=req10 +---- +new: state=doneWaiting +Intents to resolve: + key="a" txn=00000000 status=ABORTED + key="b" txn=00000000 status=ABORTED + +txn-finalized txn=txn4 status=aborted +---- + +release txn=txn4 span=c +---- +global: num=1 + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] +local: num=0 + +guard-state r=req9 +---- +new: state=doneWaiting +Intents to resolve: + key="d" txn=00000000 status=ABORTED + +print +---- +global: num=0 +local: num=0 + +# ----------------------------------------------------------------------------- +# req11 is a non-transactional request that finds a lock from a finalized txn +# when scanning. The removal of the lock before it can do resolution exposed a +# bug in the code where non-transactional requests were assumed to be active +# waiters. +# ----------------------------------------------------------------------------- + +new-request r=req11 txn=none ts=12,1 spans=w@a +---- + +scan r=req11 +---- +start-waiting: false + +add-discovered r=req11 k=a txn=txn2 +---- +global: num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + queued writers: + active: false req: 11, txn: none +local: num=0 + +txn-finalized txn=txn2 status=aborted +---- + +scan r=req11 +---- +start-waiting: true + +print +---- +global: num=0 +local: num=0 + +guard-state r=req11 +---- +new: state=doneWaiting +Intents to resolve: + key="a" txn=00000000 status=ABORTED + +dequeue r=req11 +---- +global: num=0 +local: num=0 + +clear +---- +global: num=0 +local: num=0 + +# ----------------------------------------------------------------------------- +# req12 is a read request that finds a lock from a finalized txn +# when scanning. +# ----------------------------------------------------------------------------- + +new-request r=req12 txn=none ts=12,1 spans=r@a +---- + +scan r=req12 +---- +start-waiting: false + +add-discovered r=req12 k=a txn=txn2 +---- +global: num=1 + lock: "a" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] +local: num=0 + +txn-finalized txn=txn2 status=aborted +---- + +scan r=req12 +---- +start-waiting: true + +print +---- +global: num=0 +local: num=0 + +guard-state r=req12 +---- +new: state=doneWaiting +Intents to resolve: + key="a" txn=00000000 status=ABORTED + +dequeue r=req12 +---- +global: num=0 +local: num=0 + +clear +---- +global: num=0 +local: num=0