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 87757d2efa59..ae4b1e8f6a6a 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 4631542a93f0..b82c7b847d97 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: 0.000000010,1, seq: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,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: 0.000000010,1, seq: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,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: 0.000000010,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,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: 0.000000010,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: repl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + lock: "h" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,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 0.000000010,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: 0.000000010,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: 0.000000010,1, info: repl epoch: 0, seqs: [0] + lock: "g" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + lock: "h" + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000004-0000-0000-0000-000000000000, ts: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,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: 0.000000010,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: 0.000000010,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 0.000000011,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: 0.000000010,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: 0.000000010,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,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: 0.000000010,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: 0.000000010,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 0.000000011,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: 0.000000010,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: 0.000000012,1, seq: 0 + lock: "d" + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 0.000000010,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: 0.000000010,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: 0.000000012,1, seq: 0 + lock: "d" + res: req: 5, txn: 00000001-0000-0000-0000-000000000000, ts: 0.000000012,1, seq: 0 + lock: "e" + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 0.000000010,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 60e9fdec291a..1a71193df722 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: 0.000000011,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 0.000000011,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..8856c9a15927 --- /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: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000010,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: 0.000000011,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: 0.000000010,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: 0.000000010,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: 0.000000011,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: 0.000000011,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: 0.000000010,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: 0.000000010,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: 0.000000011,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000011,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: 0.000000011,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: 0.000000010,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: 0.000000010,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: 0.000000011,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000011,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: 0.000000011,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: 0.000000010,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: 0.000000010,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: 0.000000011,1, info: unrepl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000011,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: 0.000000011,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: 0.000000010,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: 0.000000010,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000010,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: 0.000000010,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: 0.000000010,1, seq: 0 + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000011,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: 0.000000011,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: 0.000000010,1, seq: 0 + lock: "b" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, seq: 0 + lock: "c" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, seq: 0 + lock: "d" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,1, seq: 0 + lock: "e" + res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000011,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: 0.000000011,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: 0.000000011,1, info: unrepl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 0.000000011,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: 0.000000010,1, seq: 0 + lock: "c" + res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000010,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000012,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: 0.000000011,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: 0.000000012,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: 0.000000012,1, seq: 0 +local: num=0 + +dequeue r=req5 +---- +global: num=1 + lock: "a" + res: req: 6, txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000012,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000012,1, seq: 0 + lock: "b" + res: req: 7, txn: 00000000-0000-0000-0000-000000000001, ts: 0.000000012,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: 0.000000012,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: 0.000000012,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000012,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: 0.000000012,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000012,1, info: repl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 0.000000011,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: 0.000000012,1, info: repl epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000012,1, info: repl epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 0.000000011,1, info: repl epoch: 0, seqs: [0] + lock: "d" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 0.000000011,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: 0.000000012,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + lock: "b" + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 0.000000012,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + lock: "c" + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 0.000000011,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: 0.000000011,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: 0.000000011,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: 0.000000012,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: 0.000000012,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