diff --git a/pkg/kv/kvserver/batcheval/cmd_query_locks.go b/pkg/kv/kvserver/batcheval/cmd_query_locks.go index 312ec37f6f1e..a586eda1969b 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_locks.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_locks.go @@ -53,12 +53,7 @@ func QueryLocks( reply := resp.(*kvpb.QueryLocksResponse) concurrencyManager := cArgs.EvalCtx.GetConcurrencyManager() - keyScope := spanset.SpanGlobal - if keys.IsLocal(args.Key) { - keyScope = spanset.SpanLocal - } opts := concurrency.QueryLockTableOptions{ - KeyScope: keyScope, MaxLocks: h.MaxSpanRequestKeys, TargetBytes: h.TargetBytes, IncludeUncontended: args.IncludeUncontended, diff --git a/pkg/kv/kvserver/concurrency/concurrency_control.go b/pkg/kv/kvserver/concurrency/concurrency_control.go index f65627fc8d54..bbbcdae0c26a 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_control.go +++ b/pkg/kv/kvserver/concurrency/concurrency_control.go @@ -457,7 +457,6 @@ type Error = kvpb.Error // QueryLockTableOptions bundles the options for the QueryLockTableState function. type QueryLockTableOptions struct { - KeyScope spanset.SpanScope MaxLocks int64 TargetBytes int64 IncludeUncontended bool diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index f7b11128909d..79027d59f067 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -233,11 +233,11 @@ type lockTableImpl struct { // TODO(nvanbenschoten): use an atomic.Uint64. seqNum uint64 - // locks contains the btree objects (wrapped in the treeMu structure) that - // contain the actual lockState objects for a particular scope of the - // replica's key span (Global or Local). These lockState objects represent - // the individual locks in the lock table. - locks [spanset.NumSpanScope]treeMu + // locks contains the btree object (wrapped in the treeMu structure) that + // contains the actual lockState objects. These lockState objects represent + // the individual locks in the lock table. Locks on both Global and Local keys + // are stored in the same btree. + locks treeMu // maxLocks is a soft maximum on number of locks. When it is exceeded, and // subject to the dampening in lockAddMaxLocksCheckInterval, locks will be @@ -281,9 +281,7 @@ func (t *lockTableImpl) setMaxLocks(maxLocks int64) { } t.maxLocks = maxLocks t.minLocks = maxLocks / 2 - for i := 0; i < int(spanset.NumSpanScope); i++ { - t.locks[i].lockAddMaxLocksCheckInterval = uint64(lockAddMaxLocksCheckInterval) - } + t.locks.lockAddMaxLocksCheckInterval = uint64(lockAddMaxLocksCheckInterval) } // lockTableGuardImpl is an implementation of lockTableGuard. @@ -367,8 +365,8 @@ type lockTableGuardImpl struct { waitPolicy lock.WaitPolicy maxWaitQueueLength int - // Snapshots of the trees for which this request has some spans. Note that - // the lockStates in these snapshots may have been removed from + // Snapshot of the tree for which this request has some spans. Note that + // the lockStates in this snapshot may have been removed from // lockTableImpl. Additionally, it is possible that there is a new lockState // for the same key. This can result in various harmless anomalies: // - the request may hold a reservation on a lockState that is no longer @@ -384,19 +382,19 @@ type lockTableGuardImpl struct { // implementation, in comparison with eager queueing. If eager queueing // is comparable in system throughput, one can eliminate the above anomalies. // - // TODO(nvanbenschoten): should we be Reset-ing these btree snapshots when we + // TODO(nvanbenschoten): should we be Reset-ing these btree snapshot when we // Dequeue a lockTableGuardImpl? In releaseLockTableGuardImpl? // - tableSnapshot [spanset.NumSpanScope]btree + tableSnapshot btree // notRemovableLock points to the lock for which this guard has incremented // lockState.notRemovable. It will be set to nil when this guard has decremented // lockState.notRemovable. Note that: - // - notRemovableLock may no longer be in one of the btrees in lockTableImpl - // since it may have been removed due to the lock being released. This is - // harmless since the change in lock state for that lock's key (even if it - // has meanwhile been reacquired by a different request) means forward - // progress for this request, which guarantees liveness for this request. + // - notRemovableLock may no longer be the btree in lockTableImpl since it may + // have been removed due to the lock being released. This is harmless since + // the change in lock state for that lock's key (even if it has meanwhile been + // reacquired by a different request) means forward progress for this request, + // which guarantees liveness for this request. // - Multiple guards can have marked the same lock as notRemovable, which is // why lockState.notRemovable behaves like a reference count. notRemovableLock *lockState @@ -554,8 +552,7 @@ func (g *lockTableGuardImpl) CheckOptimisticNoConflicts(spanSet *spanset.SpanSet span := stepToNextSpan(g) for span != nil { startKey := span.Key - tree := g.tableSnapshot[g.ss] - iter := tree.MakeIter() + iter := g.tableSnapshot.MakeIter() ltRange := &lockState{key: startKey, endKey: span.EndKey} for iter.FirstOverlap(ltRange); iter.Valid(); iter.NextOverlap(ltRange) { l := iter.Cur() @@ -571,12 +568,7 @@ func (g *lockTableGuardImpl) CheckOptimisticNoConflicts(spanSet *spanset.SpanSet func (g *lockTableGuardImpl) IsKeyLockedByConflictingTxn( key roachpb.Key, strength lock.Strength, ) (bool, *enginepb.TxnMeta) { - ss := spanset.SpanGlobal - if keys.IsLocal(key) { - ss = spanset.SpanLocal - } - tree := g.tableSnapshot[ss] - iter := tree.MakeIter() + iter := g.tableSnapshot.MakeIter() iter.SeekGE(&lockState{key: key}) if !iter.Valid() || !iter.Cur().key.Equal(key) { // No lock on key. @@ -666,13 +658,9 @@ func (g *lockTableGuardImpl) findNextLockAfter(notify bool) { // 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 + var locksToGC []*lockState defer func() { - for i := 0; i < len(locksToGC); i++ { - if len(locksToGC[i]) > 0 { - g.lt.tryGCLocks(&g.lt.locks[i], locksToGC[i]) - } - } + g.lt.tryGCLocks(&g.lt.locks, locksToGC) }() for span != nil { @@ -680,8 +668,7 @@ func (g *lockTableGuardImpl) findNextLockAfter(notify bool) { if resumingInSameSpan { startKey = g.key } - tree := g.tableSnapshot[g.ss] - iter := tree.MakeIter() + iter := g.tableSnapshot.MakeIter() // From here on, the use of resumingInSameSpan is just a performance // optimization to deal with the interface limitation of btree that @@ -702,7 +689,7 @@ func (g *lockTableGuardImpl) findNextLockAfter(notify bool) { } wait, transitionedToFree := l.tryActiveWait(g, g.sa, notify, g.lt.clock) if transitionedToFree { - locksToGC[g.ss] = append(locksToGC[g.ss], l) + locksToGC = append(locksToGC, l) } if wait { return @@ -806,7 +793,6 @@ type lockState struct { // The key being locked and the scope of that key. This state is never // mutated. key roachpb.Key - ss spanset.SpanScope mu syncutil.Mutex // Protects everything below. @@ -1405,7 +1391,7 @@ func (l *lockState) isEmptyLock() bool { // assertEmptyLock asserts that the lockState is empty. This condition must hold // for a lock to be safely removed from the tree. If it does not hold, requests -// with stale snapshots of the btree will still be able to enter the lock's +// with a stale snapshot of the btree will still be able to enter the lock's // wait-queue, after which point they will never hear of lock updates. // REQUIRES: l.mu is locked. func (l *lockState) assertEmptyLock() { @@ -2550,21 +2536,15 @@ func (t *lockTableImpl) newGuardForReq(req Request) *lockTableGuardImpl { } func (t *lockTableImpl) doSnapshotForGuard(g *lockTableGuardImpl) { - for ss := spanset.SpanScope(0); ss < spanset.NumSpanScope; ss++ { - for sa := spanset.SpanAccess(0); sa < spanset.NumSpanAccess; sa++ { - if len(g.spans.GetSpans(sa, ss)) > 0 { - // Since the spans are constant for a request, every call to - // ScanAndEnqueue for that request will execute the following code - // for the same SpanScope(s). Any SpanScope for which this code does - // not execute will always have an empty snapshot. - t.locks[ss].mu.RLock() - g.tableSnapshot[ss].Reset() - g.tableSnapshot[ss] = t.locks[ss].Clone() - t.locks[ss].mu.RUnlock() - break - } - } + if g.spans.Empty() { + // A request with no lock spans has an empty snapshot as it doesn't need + // one. + return } + t.locks.mu.RLock() + g.tableSnapshot.Reset() + g.tableSnapshot = t.locks.Clone() + t.locks.mu.RUnlock() } // Dequeue implements the lockTable interface. @@ -2585,18 +2565,14 @@ func (t *lockTableImpl) Dequeue(guard lockTableGuard) { candidateLocks = append(candidateLocks, l) } g.mu.Unlock() - var locksToGC [spanset.NumSpanScope][]*lockState + var locksToGC []*lockState for _, l := range candidateLocks { if gc := l.requestDone(g); gc { - locksToGC[l.ss] = append(locksToGC[l.ss], l) + locksToGC = append(locksToGC, l) } } - for i := 0; i < len(locksToGC); i++ { - if len(locksToGC[i]) > 0 { - t.tryGCLocks(&t.locks[i], locksToGC[i]) - } - } + t.tryGCLocks(&t.locks, locksToGC) } // AddDiscoveredLock implements the lockTable interface. @@ -2646,7 +2622,7 @@ func (t *lockTableImpl) AddDiscoveredLock( } g := guard.(*lockTableGuardImpl) key := intent.Key - sa, ss, err := findAccessInSpans(key, g.spans) + sa, _, err := findAccessInSpans(key, g.spans) if err != nil { return false, err } @@ -2659,19 +2635,18 @@ func (t *lockTableImpl) AddDiscoveredLock( } } var l *lockState - tree := &t.locks[ss] - tree.mu.Lock() - iter := tree.MakeIter() + t.locks.mu.Lock() + iter := t.locks.MakeIter() iter.FirstOverlap(&lockState{key: key}) checkMaxLocks := false if !iter.Valid() { var lockSeqNum uint64 - lockSeqNum, checkMaxLocks = tree.nextLockSeqNum() - l = &lockState{id: lockSeqNum, key: key, ss: ss} + lockSeqNum, checkMaxLocks = t.locks.nextLockSeqNum() + l = &lockState{id: lockSeqNum, key: key} l.queuedWriters.Init() l.waitingReaders.Init() - tree.Set(l) - atomic.AddInt64(&tree.numLocks, 1) + t.locks.Set(l) + atomic.AddInt64(&t.locks.numLocks, 1) } else { l = iter.Cur() } @@ -2687,7 +2662,7 @@ func (t *lockTableImpl) AddDiscoveredLock( err = l.discoveredLock(&intent.Txn, intent.Txn.WriteTimestamp, g, sa, notRemovableLock, g.lt.clock) // Can't release tree.mu until call l.discoveredLock() since someone may // find an empty lock and remove it from the tree. - tree.mu.Unlock() + t.locks.mu.Unlock() if checkMaxLocks { t.checkMaxLocksAndTryClear() } @@ -2707,18 +2682,13 @@ func (t *lockTableImpl) AcquireLock( if strength != lock.Exclusive { return errors.AssertionFailedf("lock strength not Exclusive") } - ss := spanset.SpanGlobal - if keys.IsLocal(key) { - ss = spanset.SpanLocal - } var l *lockState - tree := &t.locks[ss] - tree.mu.Lock() + t.locks.mu.Lock() // Can't release tree.mu until call l.acquireLock() since someone may find // an empty lock and remove it from the tree. If we expect that lockState // will already be in tree we can optimize this by first trying with a // tree.mu.RLock(). - iter := tree.MakeIter() + iter := t.locks.MakeIter() iter.FirstOverlap(&lockState{key: key}) checkMaxLocks := false if !iter.Valid() { @@ -2729,16 +2699,16 @@ func (t *lockTableImpl) AcquireLock( // running into the maxLocks limit is somewhat crude. Treating the // data-structure as a bounded cache with eviction guided by contention // would be better. - tree.mu.Unlock() + t.locks.mu.Unlock() return nil } var lockSeqNum uint64 - lockSeqNum, checkMaxLocks = tree.nextLockSeqNum() - l = &lockState{id: lockSeqNum, key: key, ss: ss} + lockSeqNum, checkMaxLocks = t.locks.nextLockSeqNum() + l = &lockState{id: lockSeqNum, key: key} l.queuedWriters.Init() l.waitingReaders.Init() - tree.Set(l) - atomic.AddInt64(&tree.numLocks, 1) + t.locks.Set(l) + atomic.AddInt64(&t.locks.numLocks, 1) } else { l = iter.Cur() if durability == lock.Replicated && l.tryFreeLockOnReplicatedAcquire() { @@ -2749,14 +2719,14 @@ func (t *lockTableImpl) AcquireLock( // TODO(sumeer): now that limited scans evaluate optimistically, we // should consider removing this hack. But see the comment in the // preceding block about maxLocks. - tree.Delete(l) - tree.mu.Unlock() - atomic.AddInt64(&tree.numLocks, -1) + t.locks.Delete(l) + t.locks.mu.Unlock() + atomic.AddInt64(&t.locks.numLocks, -1) return nil } } err := l.acquireLock(strength, durability, txn, txn.WriteTimestamp, t.clock) - tree.mu.Unlock() + t.locks.mu.Unlock() if checkMaxLocks { t.checkMaxLocksAndTryClear() @@ -2765,10 +2735,7 @@ func (t *lockTableImpl) AcquireLock( } func (t *lockTableImpl) checkMaxLocksAndTryClear() { - var totalLocks int64 - for i := 0; i < len(t.locks); i++ { - totalLocks += atomic.LoadInt64(&t.locks[i].numLocks) - } + totalLocks := atomic.LoadInt64(&t.locks.numLocks) if totalLocks > t.maxLocks { numToClear := totalLocks - t.minLocks t.tryClearLocks(false /* force */, int(numToClear)) @@ -2776,11 +2743,7 @@ func (t *lockTableImpl) checkMaxLocksAndTryClear() { } func (t *lockTableImpl) lockCountForTesting() int64 { - var totalLocks int64 - for i := 0; i < len(t.locks); i++ { - totalLocks += atomic.LoadInt64(&t.locks[i].numLocks) - } - return totalLocks + return atomic.LoadInt64(&t.locks.numLocks) } // tryClearLocks attempts to clear locks. @@ -2794,10 +2757,9 @@ func (t *lockTableImpl) tryClearLocks(force bool, numToClear int) { done := false clearCount := 0 for i := 0; i < int(spanset.NumSpanScope) && !done; i++ { - tree := &t.locks[i] - tree.mu.Lock() + t.locks.mu.Lock() var locksToClear []*lockState - iter := tree.MakeIter() + iter := t.locks.MakeIter() for iter.First(); iter.Valid(); iter.Next() { l := iter.Cur() if l.tryClearLock(force) { @@ -2809,16 +2771,16 @@ func (t *lockTableImpl) tryClearLocks(force bool, numToClear int) { } } } - atomic.AddInt64(&tree.numLocks, int64(-len(locksToClear))) - if tree.Len() == len(locksToClear) { + atomic.AddInt64(&t.locks.numLocks, int64(-len(locksToClear))) + if t.locks.Len() == len(locksToClear) { // Fast-path full clear. - tree.Reset() + t.locks.Reset() } else { for _, l := range locksToClear { - tree.Delete(l) + t.locks.Delete(l) } } - tree.mu.Unlock() + t.locks.mu.Unlock() } } @@ -2847,6 +2809,9 @@ func findAccessInSpans( // Tries to GC locks that were previously known to have become empty. func (t *lockTableImpl) tryGCLocks(tree *treeMu, locks []*lockState) { + if len(locks) == 0 { + return // bail early + } tree.mu.Lock() defer tree.mu.Unlock() for _, l := range locks { @@ -2885,11 +2850,6 @@ func (t *lockTableImpl) updateLockInternal(up *roachpb.LockUpdate) (heldByTxn bo // then it might update a few locks, but they will quickly be cleared. span := up.Span - ss := spanset.SpanGlobal - if keys.IsLocal(span.Key) { - ss = spanset.SpanLocal - } - tree := &t.locks[ss] var locksToGC []*lockState heldByTxn = false changeFunc := func(l *lockState) { @@ -2899,8 +2859,8 @@ func (t *lockTableImpl) updateLockInternal(up *roachpb.LockUpdate) (heldByTxn bo locksToGC = append(locksToGC, l) } } - tree.mu.RLock() - iter := tree.MakeIter() + t.locks.mu.RLock() + iter := t.locks.MakeIter() ltRange := &lockState{key: span.Key, endKey: span.EndKey} for iter.FirstOverlap(ltRange); iter.Valid(); iter.NextOverlap(ltRange) { changeFunc(iter.Cur()) @@ -2909,11 +2869,9 @@ func (t *lockTableImpl) updateLockInternal(up *roachpb.LockUpdate) (heldByTxn bo break } } - tree.mu.RUnlock() + t.locks.mu.RUnlock() - if len(locksToGC) > 0 { - t.tryGCLocks(tree, locksToGC) - } + t.tryGCLocks(&t.locks, locksToGC) return heldByTxn } @@ -2992,13 +2950,11 @@ func (t *lockTableImpl) QueryLockTableState( } // Grab tree snapshot to avoid holding read lock during iteration. - var snap btree - { - tree := &t.locks[opts.KeyScope] - tree.mu.RLock() - snap = tree.Clone() - tree.mu.RUnlock() - } + t.locks.mu.RLock() + snap := t.locks.Clone() + t.locks.mu.RUnlock() + // Reset snapshot to free resources. + defer snap.Reset() now := t.clock.PhysicalTime() @@ -3049,25 +3005,18 @@ func (t *lockTableImpl) QueryLockTableState( // Metrics implements the lockTable interface. func (t *lockTableImpl) Metrics() LockTableMetrics { var m LockTableMetrics - for i := 0; i < len(t.locks); i++ { - // Grab tree snapshot to avoid holding read lock during iteration. - var snap btree - { - tree := &t.locks[i] - tree.mu.RLock() - snap = tree.Clone() - tree.mu.RUnlock() - } - - // Iterate and compute metrics. - now := t.clock.PhysicalTime() - iter := snap.MakeIter() - for iter.First(); iter.Valid(); iter.Next() { - iter.Cur().addToMetrics(&m, now) - } + // Grab tree snapshot to avoid holding read lock during iteration. + t.locks.mu.RLock() + snap := t.locks.Clone() + t.locks.mu.RUnlock() + // Reset snapshot to free resources. + defer snap.Reset() - // Reset snapshot to free resources. - snap.Reset() + // Iterate and compute metrics. + now := t.clock.PhysicalTime() + iter := snap.MakeIter() + for iter.First(); iter.Valid(); iter.Next() { + iter.Cur().addToMetrics(&m, now) } return m } @@ -3075,19 +3024,15 @@ func (t *lockTableImpl) Metrics() LockTableMetrics { // String implements the lockTable interface. func (t *lockTableImpl) String() string { var sb redact.StringBuilder - for i := 0; i < len(t.locks); i++ { - tree := &t.locks[i] - scope := spanset.SpanScope(i).String() - tree.mu.RLock() - sb.Printf("%s: num=%d\n", scope, atomic.LoadInt64(&tree.numLocks)) - iter := tree.MakeIter() - for iter.First(); iter.Valid(); iter.Next() { - l := iter.Cur() - l.mu.Lock() - l.safeFormat(&sb, &t.finalizedTxnCache) - l.mu.Unlock() - } - tree.mu.RUnlock() + t.locks.mu.RLock() + sb.Printf("num=%d\n", atomic.LoadInt64(&t.locks.numLocks)) + iter := t.locks.MakeIter() + for iter.First(); iter.Valid(); iter.Next() { + l := iter.Cur() + l.mu.Lock() + l.safeFormat(&sb, &t.finalizedTxnCache) + l.mu.Unlock() } + t.locks.mu.RUnlock() return sb.String() } diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 1fdf427269e8..d7ad7826cc34 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -805,7 +805,7 @@ func TestLockTableMaxLocks(t *testing.T) { require.NoError(t, err) // The 6 notRemovable locks remain. require.Equal(t, int64(6), lt.lockCountForTesting()) - require.Equal(t, int64(101), int64(lt.locks[spanset.SpanGlobal].lockIDSeqNum)) + require.Equal(t, int64(101), int64(lt.locks.lockIDSeqNum)) // Add another discovered lock, to trigger tryClearLocks. added, err = lt.AddDiscoveredLock( &roachpb.Intent{Intent_SingleKeySpan: roachpb.Intent_SingleKeySpan{Key: keys[9*20+11]}}, @@ -814,12 +814,12 @@ func TestLockTableMaxLocks(t *testing.T) { require.NoError(t, err) // Still the 6 notRemovable locks remain. require.Equal(t, int64(6), lt.lockCountForTesting()) - require.Equal(t, int64(102), int64(lt.locks[spanset.SpanGlobal].lockIDSeqNum)) + require.Equal(t, int64(102), int64(lt.locks.lockIDSeqNum)) // Two more guards are dequeued, so we are down to 4 notRemovable locks. lt.Dequeue(guards[4]) lt.Dequeue(guards[5]) // Bump up the enforcement interval manually. - lt.locks[spanset.SpanGlobal].lockAddMaxLocksCheckInterval = 2 + lt.locks.lockAddMaxLocksCheckInterval = 2 // Add another discovered lock. added, err = lt.AddDiscoveredLock( &roachpb.Intent{Intent_SingleKeySpan: roachpb.Intent_SingleKeySpan{Key: keys[9*20+12]}}, @@ -837,7 +837,7 @@ func TestLockTableMaxLocks(t *testing.T) { // Now enforcement is done, so only 4 remain. require.Equal(t, int64(4), lt.lockCountForTesting()) // Bump down the enforcement interval manually, and bump up minLocks - lt.locks[spanset.SpanGlobal].lockAddMaxLocksCheckInterval = 1 + lt.locks.lockAddMaxLocksCheckInterval = 1 lt.minLocks = 2 // Three more guards dequeued. lt.Dequeue(guards[6]) diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic index 8f0cb2071046..8522bba6ab25 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/basic @@ -56,10 +56,9 @@ on-lock-acquired req=req2 key=k debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 12.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 finish req=req2 ---- @@ -67,10 +66,9 @@ finish req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 12.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 reset ---- @@ -78,8 +76,7 @@ reset # Demonstrate that 'reset' clears the lock table. debug-lock-table ---- -global: num=0 -local: num=0 +num=0 # ------------------------------------------------------------- # 1. Acquire a lock @@ -136,8 +133,7 @@ on-txn-updated txn=txn2 status=committed debug-lock-table ---- -global: num=0 -local: num=0 +num=0 new-request name=req4 txn=txn1 ts=10,1 put key=k value=v 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 bd57ad431e84..99ac4f4629ff 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents @@ -35,7 +35,7 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=10 +num=10 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "b" @@ -56,7 +56,6 @@ global: num=10 holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "j" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req1 ---- @@ -71,7 +70,7 @@ sequence req=req1 debug-lock-table ---- -global: num=10 +num=10 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] waiting readers: @@ -95,7 +94,6 @@ global: num=10 holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "j" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 debug-advance-clock ts=123 ---- @@ -125,8 +123,7 @@ on-txn-updated txn=txn2 status=aborted debug-lock-table ---- -global: num=0 -local: num=0 +num=0 finish req=req1 ---- @@ -165,12 +162,11 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 -local: num=0 sequence req=req1 ---- @@ -203,14 +199,13 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=2 +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 [holder finalized: committed] epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 -local: num=0 sequence req=req1 ---- @@ -232,7 +227,7 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=3 +num=3 lock: "a" res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" @@ -241,7 +236,6 @@ global: num=3 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 sequence req=req1 ---- @@ -258,14 +252,13 @@ sequence req=req1 debug-lock-table ---- -global: num=3 +num=3 lock: "a" res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "c" res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 -local: num=0 finish req=req1 ---- @@ -305,12 +298,11 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=2 +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 @@ -338,7 +330,7 @@ finish req=req2 debug-lock-table ---- -global: num=4 +num=4 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] lock: "b" @@ -347,7 +339,6 @@ global: num=4 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 ---- @@ -362,7 +353,7 @@ sequence req=req1 debug-lock-table ---- -global: num=4 +num=4 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] waiting readers: @@ -374,7 +365,6 @@ global: num=4 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 debug-advance-clock ts=123 ---- @@ -393,8 +383,7 @@ on-txn-updated txn=txn2 status=aborted debug-lock-table ---- -global: num=0 -local: num=0 +num=0 finish req=req1 ---- @@ -452,7 +441,7 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=3 +num=3 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -465,7 +454,6 @@ global: num=3 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 ---- @@ -480,7 +468,7 @@ sequence req=req1 debug-lock-table ---- -global: num=3 +num=3 lock: "c" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -494,7 +482,6 @@ global: num=3 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 @@ -536,7 +523,7 @@ finish req=req4 debug-lock-table ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" @@ -554,7 +541,6 @@ global: num=5 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 @@ -574,7 +560,7 @@ sequence req=req2 debug-lock-table ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -595,7 +581,6 @@ global: num=5 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 debug-advance-clock ts=123 ---- @@ -620,7 +605,7 @@ on-txn-updated txn=txn3 status=aborted debug-lock-table ---- -global: num=4 +num=4 lock: "b" holder: txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -637,7 +622,6 @@ global: num=4 queued writers: active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 -local: num=0 debug-advance-clock ts=123 ---- @@ -658,7 +642,7 @@ on-txn-updated txn=txn4 status=aborted debug-lock-table ---- -global: num=3 +num=3 lock: "c" res: req: 5, txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,1, seq: 0 lock: "d" @@ -668,7 +652,6 @@ global: num=3 queued writers: active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 -local: num=0 finish req=req2 ---- @@ -692,8 +675,7 @@ finish req=req1 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 reset namespace ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table index e04f178c340f..2be8231b887c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table @@ -32,10 +32,9 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req1 ---- @@ -50,13 +49,12 @@ sequence req=req1 debug-lock-table ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] waiting readers: req: 1, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 1 -local: num=0 debug-advance-clock ts=123 ---- @@ -75,8 +73,7 @@ on-txn-updated txn=txn2 status=aborted debug-lock-table ---- -global: num=0 -local: num=0 +num=0 finish req=req1 ---- @@ -119,8 +116,7 @@ handle-write-intent-error req=req2 lease-seq=1 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 sequence req=req2 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks index 6ccd50aa8b7d..187a981cf8f8 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/deadlocks @@ -81,14 +81,13 @@ finish req=req3w debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000002-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: unrepl epoch: 0, seqs: [0] -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -144,7 +143,7 @@ sequence req=req3r debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -160,7 +159,6 @@ global: num=3 waiting readers: req: 5, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 5 -local: num=0 # Break the deadlock by aborting txn1. on-txn-updated txn=txn1 status=aborted @@ -293,14 +291,13 @@ finish req=req3w debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000002-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: unrepl epoch: 0, seqs: [0] -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -371,7 +368,7 @@ sequence req=req3w2 debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -388,7 +385,6 @@ global: num=3 queued writers: active: true req: 12, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 12 -local: num=0 # Break the deadlock by aborting txn1. on-txn-updated txn=txn1 status=aborted @@ -539,14 +535,13 @@ finish req=req3w debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000002-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: unrepl epoch: 0, seqs: [0] -local: num=0 new-request name=req4w txn=txn4 ts=10,1 put key=b value=v2 @@ -576,7 +571,7 @@ on-txn-updated txn=txn2 status=committed debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" @@ -586,7 +581,6 @@ global: num=3 queued writers: active: true req: 17, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 17 -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -627,7 +621,7 @@ sequence req=req3w2 debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -643,7 +637,6 @@ global: num=3 queued writers: active: true req: 17, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 17 -local: num=0 # Break the deadlock by aborting txn4. on-txn-updated txn=txn4 status=aborted @@ -777,14 +770,13 @@ finish req=req3w debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000002-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: unrepl epoch: 0, seqs: [0] -local: num=0 new-request name=req4w txn=txn4 ts=10,1 put key=b value=v2 @@ -814,7 +806,7 @@ on-txn-updated txn=txn2 status=committed debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" @@ -824,7 +816,6 @@ global: num=3 queued writers: active: true req: 23, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 23 -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -865,7 +856,7 @@ sequence req=req3w2 debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -881,7 +872,6 @@ global: num=3 queued writers: active: true req: 23, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 23 -local: num=0 # Break the deadlock by aborting txn1. on-txn-updated txn=txn1 status=aborted @@ -1022,14 +1012,13 @@ finish req=req3w debug-lock-table ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000002-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: unrepl epoch: 0, seqs: [0] -local: num=0 new-request name=req4w txn=txn4 ts=10,1 put key=a value=v2 @@ -1091,7 +1080,7 @@ on-txn-updated txn=txn2 status=committed debug-lock-table ---- -global: num=3 +num=3 lock: "a" res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 lock: "b" @@ -1104,7 +1093,6 @@ global: num=3 queued writers: active: true req: 29, txn: 00000005-0000-0000-0000-000000000000 distinguished req: 29 -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -1130,7 +1118,7 @@ sequence req=req3w2 debug-lock-table ---- -global: num=3 +num=3 lock: "a" res: req: 30, txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: @@ -1146,7 +1134,6 @@ global: num=3 queued writers: active: true req: 29, txn: 00000005-0000-0000-0000-000000000000 distinguished req: 29 -local: num=0 # Break the deadlock by aborting txn4. on-txn-updated txn=txn4 status=aborted diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race index 759ff68bde33..a576a81d4ebc 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discover_lock_after_lease_race @@ -78,8 +78,7 @@ finish req=req1 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 # -------------------------------- # Setup complete, test starts here @@ -101,8 +100,7 @@ on-lease-updated leaseholder=false lease-seq=2 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 # The following series of events takes place on the new leaseholder. # @@ -138,10 +136,9 @@ handle-write-intent-error req=req4 lease-seq=3 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req4 ---- @@ -162,13 +159,12 @@ handle-write-intent-error req=req2 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] waiting readers: req: 3, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 3 -local: num=0 sequence req=req2 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock index b2651e3a0c85..13b5317053cf 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock @@ -27,10 +27,9 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req1 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout index e3aad4fe7299..4c546c3f219f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout @@ -77,7 +77,7 @@ sequence req=req3 debug-lock-table ---- -global: num=3 +num=3 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "k2" @@ -87,7 +87,6 @@ global: num=3 distinguished req: 3 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Read-only request with lock timeout hits lock. The request @@ -139,7 +138,7 @@ finish req=reqTimeout1 debug-lock-table ---- -global: num=2 +num=2 lock: "k2" res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 lock: "k3" @@ -147,7 +146,6 @@ global: num=2 queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 -local: num=0 # ------------------------------------------------------------- # Read-write request with lock timeout hits reservation @@ -206,7 +204,7 @@ sequence req=reqTimeout3 debug-lock-table ---- -global: num=3 +num=3 lock: "k2" res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 lock: "k3" @@ -216,7 +214,6 @@ global: num=3 distinguished req: 3 lock: "k4" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Read-only request with lock timeout discovers abandoned @@ -272,10 +269,9 @@ finish req=req3 debug-lock-table ---- -global: num=1 +num=1 lock: "k4" 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/concurrency_manager/optimistic b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic index f9e399c75752..a6c99419398f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/optimistic @@ -21,10 +21,9 @@ on-lock-acquired req=req1 key=d debug-lock-table ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 finish req=req1 ---- @@ -44,10 +43,9 @@ sequence req=req2 eval-kind=opt debug-lock-table ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # When checking with a span that does not include the existing lock, there is # no conflict. @@ -74,10 +72,9 @@ sequence req=req3 eval-kind=opt debug-lock-table ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # Wider span for req3 has a conflict. check-opt-no-conflicts req=req3 @@ -130,8 +127,7 @@ sequence req=req4 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 new-request name=req5 txn=txn2 ts=12,1 scan key=a endkey=e diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority index f9da4e710abb..6917fc936faa 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/priority @@ -103,7 +103,7 @@ finish req=req3 debug-lock-table ---- -global: num=6 +num=6 lock: "kHigh1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kHigh2" @@ -116,7 +116,6 @@ global: num=6 holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kNormal2" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Push (timestamp) the low priority txn using: @@ -176,7 +175,7 @@ finish req=req5 debug-lock-table ---- -global: num=6 +num=6 lock: "kHigh1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kHigh2" @@ -189,7 +188,6 @@ global: num=6 holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kNormal2" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Push (abort) the low priority txn using: @@ -255,7 +253,7 @@ finish req=req7 debug-lock-table ---- -global: num=5 +num=5 lock: "kHigh1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kHigh2" @@ -266,7 +264,6 @@ global: num=5 holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kNormal2" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Push (timestamp) the normal priority txn using: @@ -326,7 +323,7 @@ finish req=req9 debug-lock-table ---- -global: num=5 +num=5 lock: "kHigh1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kHigh2" @@ -337,7 +334,6 @@ global: num=5 holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "kNormal2" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Push (abort) the normal priority txn using: @@ -403,7 +399,7 @@ finish req=req11 debug-lock-table ---- -global: num=4 +num=4 lock: "kHigh1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "kHigh2" @@ -412,7 +408,6 @@ global: num=4 holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl [holder finalized: aborted] epoch: 0, seqs: [0] lock: "kNormal1" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl [holder finalized: aborted] epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Push (timestamp) the high priority txn using: @@ -476,7 +471,7 @@ finish req=req13 debug-lock-table ---- -global: num=4 +num=4 lock: "kHigh1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "kHigh2" @@ -485,7 +480,6 @@ global: num=4 holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl [holder finalized: aborted] epoch: 0, seqs: [0] lock: "kNormal1" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl [holder finalized: aborted] epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Push (abort) the high priority txn using: @@ -553,14 +547,13 @@ finish req=req15 debug-lock-table ---- -global: num=3 +num=3 lock: "kHigh1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl [holder finalized: committed] epoch: 0, seqs: [0] lock: "kLow1" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl [holder finalized: aborted] epoch: 0, seqs: [0] lock: "kNormal1" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,2, info: unrepl [holder finalized: aborted] epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Scan across keyspace to clear out all aborted locks. @@ -583,8 +576,7 @@ finish req=req16 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 reset ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded index c2327627d355..b59f54358485 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/queue_length_exceeded @@ -87,7 +87,7 @@ sequence req=req4 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -95,7 +95,6 @@ global: num=1 active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 active: true req: 4, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 2 -local: num=0 # ------------------------------------------------------------- # Read-only request runs into long lock wait-queue. Waits for @@ -168,14 +167,13 @@ finish req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 active: true req: 4, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 3 -local: num=0 # ------------------------------------------------------------- # Read-write request runs into long lock wait-queue. Instead of diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener index 96b5c94f8fd5..2d65e5225058 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener @@ -77,12 +77,11 @@ finish req=req1 debug-lock-table ---- -global: num=2 +num=2 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "k2" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -100,7 +99,7 @@ sequence req=req2 debug-lock-table ---- -global: num=2 +num=2 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -108,7 +107,6 @@ global: num=2 distinguished req: 2 lock: "k2" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # Replica loses lease. on-lease-updated leaseholder=false lease-seq=2 @@ -122,8 +120,7 @@ on-lease-updated leaseholder=false lease-seq=2 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 handle-write-intent-error req=req2 lease-seq=1 intent txn=txn1 key=k @@ -133,8 +130,7 @@ handle-write-intent-error req=req2 lease-seq=1 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 sequence req=req2 ---- @@ -166,12 +162,11 @@ handle-write-intent-error req=req2 lease-seq=2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 3, txn: 00000002-0000-0000-0000-000000000000 -local: num=0 sequence req=req2 ---- @@ -215,10 +210,9 @@ finish req=reqRes1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" res: req: 3, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 -local: num=0 on-lock-acquired req=req2 key=k ---- @@ -226,10 +220,9 @@ on-lock-acquired req=req2 key=k debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 finish req=req2 ---- @@ -246,8 +239,7 @@ on-lease-updated leaseholder=true lease-seq=4 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 sequence req=req3 ---- @@ -268,8 +260,7 @@ handle-write-intent-error req=req3 lease-seq=2 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 sequence req=req3 ---- @@ -285,12 +276,11 @@ handle-write-intent-error req=req3 lease-seq=4 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 4, txn: 00000003-0000-0000-0000-000000000000 -local: num=0 sequence req=req3 ---- @@ -329,10 +319,9 @@ finish req=reqRes2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" res: req: 4, txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 -local: num=0 on-lock-acquired req=req3 key=k ---- @@ -340,10 +329,9 @@ on-lock-acquired req=req3 key=k debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 finish req=req3 ---- @@ -398,10 +386,9 @@ finish req=req1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -419,13 +406,12 @@ sequence req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 6, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 6 -local: num=0 on-split ---- @@ -438,8 +424,7 @@ on-split debug-lock-table ---- -global: num=0 -local: num=0 +num=0 handle-write-intent-error req=req2 lease-seq=1 intent txn=txn1 key=k @@ -448,12 +433,11 @@ handle-write-intent-error req=req2 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 6, txn: 00000002-0000-0000-0000-000000000000 -local: num=0 sequence req=req2 ---- @@ -492,10 +476,9 @@ finish req=reqRes1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" res: req: 6, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 -local: num=0 on-lock-acquired req=req2 key=k ---- @@ -503,10 +486,9 @@ on-lock-acquired req=req2 key=k debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 finish req=req2 ---- @@ -575,10 +557,9 @@ finish req=req1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -603,13 +584,12 @@ sequence req=req3 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 8, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 8 -local: num=0 on-merge ---- @@ -622,8 +602,7 @@ on-merge debug-lock-table ---- -global: num=0 -local: num=0 +num=0 on-lock-acquired req=req3 key=k2 ---- @@ -631,8 +610,7 @@ on-lock-acquired req=req3 key=k2 debug-lock-table ---- -global: num=0 -local: num=0 +num=0 finish req=req3 ---- @@ -674,12 +652,11 @@ handle-write-intent-error req=req2 lease-seq=2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 10, txn: 00000002-0000-0000-0000-000000000000 -local: num=0 sequence req=req2 ---- @@ -718,10 +695,9 @@ finish req=reqRes1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" res: req: 10, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 -local: num=0 on-lock-acquired req=req2 key=k ---- @@ -729,10 +705,9 @@ on-lock-acquired req=req2 key=k debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 finish req=req2 ---- @@ -787,10 +762,9 @@ finish req=req1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -808,13 +782,12 @@ sequence req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 12, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 12 -local: num=0 on-snapshot-applied ---- @@ -827,8 +800,7 @@ on-snapshot-applied debug-lock-table ---- -global: num=0 -local: num=0 +num=0 handle-write-intent-error req=req2 lease-seq=1 intent txn=txn1 key=k @@ -837,12 +809,11 @@ handle-write-intent-error req=req2 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 12, txn: 00000002-0000-0000-0000-000000000000 -local: num=0 sequence req=req2 ---- @@ -881,10 +852,9 @@ finish req=reqRes1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" res: req: 12, txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 -local: num=0 on-lock-acquired req=req2 key=k ---- @@ -892,10 +862,9 @@ on-lock-acquired req=req2 key=k debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 finish req=req2 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty index 482c4c1e53f9..e099b7bea4a1 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty @@ -29,10 +29,9 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req1 ---- @@ -96,10 +95,9 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 100.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req1 ---- @@ -166,10 +164,9 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 100.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req1 ---- @@ -242,10 +239,9 @@ handle-write-intent-error req=req1 lease-seq=1 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 14.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 sequence req=req1 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update index be8a1e4816f7..6a55b87a2047 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update @@ -56,13 +56,12 @@ sequence req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 2, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 2 -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -84,10 +83,9 @@ finish req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0] -local: num=0 # Issue another write to the same key for txn1 at its initial # timestamp. The timestamp in the lock table does not regress. @@ -113,10 +111,9 @@ finish req=req3 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0, 1] -local: num=0 reset namespace ---- @@ -178,13 +175,12 @@ sequence req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 5, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 5 -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -206,10 +202,9 @@ finish req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0] -local: num=0 # The txn restarts at a new timestamp, but below the pushed # timestamp. It re-issues the same write at the new epoch. The @@ -239,10 +234,9 @@ finish req=req3 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 1, seqs: [0] -local: num=0 reset namespace ---- @@ -309,13 +303,12 @@ sequence req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-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 -local: num=0 # -------------------------------- # Setup complete, test starts here @@ -337,10 +330,9 @@ finish req=req2 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 12.000000000,2, info: unrepl epoch: 0, seqs: [0] -local: num=0 # Issue another write to the same key for txn1 at its initial timestamp, # this time with a replicated durability. The timestamp in the lock @@ -387,13 +379,12 @@ finish req=req3 debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [1], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 9, txn: none distinguished req: 9 -local: num=0 # Finish off txn1. Not needed once we can get rid of req4. on-txn-updated txn=txn1 status=committed diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere index 1783b5e180fa..c5212050ac32 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere @@ -67,13 +67,12 @@ sequence req=reqWaiter debug-lock-table ---- -global: num=1 +num=1 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 2 -local: num=0 sequence req=reqSecondLock ---- @@ -173,8 +172,7 @@ finish req=reqThreeKeyWriter debug-lock-table ---- -global: num=0 -local: num=0 +num=0 sequence req=reqTwoKeyWaiter ---- @@ -211,7 +209,7 @@ sequence req=reqTwoKeyWaiter debug-lock-table ---- -global: num=2 +num=2 lock: "k1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] waiting readers: @@ -219,7 +217,6 @@ global: num=2 distinguished req: 5 lock: "k2" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 # Simulate that the replicated locks were discovered, so they are added to the # lock table. Keys "k1" and "k2" were previously discovered, but "k3" is new. @@ -243,14 +240,13 @@ sequence req=reqThreeKeyWaiter debug-lock-table ---- -global: num=1 +num=1 lock: "k1" holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] waiting readers: req: 6, txn: 00000002-0000-0000-0000-000000000000 req: 5, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 5 -local: num=0 # Before #99635 was fixed, reqTwoKeyWaiter would move on to waiting on key k2 # and get stuck in lockTableWaiterImpl.WaitOn. Even after it resolved the intent 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 5dd811f7ca23..8df3fef6527c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error @@ -77,7 +77,7 @@ sequence req=req3 debug-lock-table ---- -global: num=3 +num=3 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "k2" @@ -87,7 +87,6 @@ global: num=3 distinguished req: 3 lock: "k3" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Read-only request with WaitPolicy_Error hits lock. The request @@ -141,7 +140,7 @@ finish req=reqNoWait1 debug-lock-table ---- -global: num=2 +num=2 lock: "k2" res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 lock: "k3" @@ -149,7 +148,6 @@ global: num=2 queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 -local: num=0 # ------------------------------------------------------------- # Read-write request with WaitPolicy_Error hits reservation @@ -206,7 +204,7 @@ sequence req=reqNoWait3 debug-lock-table ---- -global: num=3 +num=3 lock: "k2" res: req: 3, txn: 00000003-0000-0000-0000-000000000000, ts: 11.000000000,0, seq: 0 lock: "k3" @@ -216,7 +214,6 @@ global: num=3 distinguished req: 3 lock: "k4" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Read-only request with WaitPolicy_Error discovers abandoned @@ -275,10 +272,9 @@ finish req=req3 debug-lock-table ---- -global: num=1 +num=1 lock: "k4" 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/concurrency_manager/wait_policy_skip b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip index 4ff03f35e57a..5a1e49f36305 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_skip @@ -105,7 +105,7 @@ on-txn-updated txn=txn3 status=aborted debug-lock-table ---- -global: num=4 +num=4 lock: "k" holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "k2" @@ -114,7 +114,6 @@ global: num=4 holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "k4" res: req: 4, txn: 00000004-0000-0000-0000-000000000000, ts: 13.000000000,0, seq: 0 -local: num=0 # ------------------------------------------------------------- # Read-only request with WaitPolicy_Skip hits lock sequences diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self index 3aa835923c69..4fe814c78268 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_self @@ -99,14 +99,13 @@ on-txn-updated txn=txnOld status=committed debug-lock-table ---- -global: num=1 +num=1 lock: "k" res: req: 2, txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 active: true req: 4, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 3 -local: num=0 # This is the interesting step - we see reqTxn2 announce that it conflicted with diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency index 9d67c43308eb..c363274841c3 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_idempotency @@ -23,17 +23,15 @@ start-waiting: false acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1] -local: num=0 dequeue r=req1 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1] -local: num=0 new-txn txn=txn1 ts=10,1 epoch=0 seq=2 ---- @@ -47,17 +45,15 @@ start-waiting: false acquire r=req2 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2] -local: num=0 dequeue r=req2 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2] -local: num=0 new-txn txn=txn1 ts=10,1 epoch=0 seq=4 ---- @@ -71,17 +67,15 @@ start-waiting: false acquire r=req3 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] -local: num=0 dequeue r=req3 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] -local: num=0 # ------------------------------------------------------------- # Re-Acquire lock with sequence number 4 @@ -99,17 +93,15 @@ start-waiting: false acquire r=req3 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] -local: num=0 dequeue r=req3 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] -local: num=0 # ------------------------------------------------------------- # Re-Acquire lock with sequence number 2 @@ -127,17 +119,15 @@ start-waiting: false acquire r=req4 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] -local: num=0 dequeue r=req4 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 4] -local: num=0 # ------------------------------------------------------------- # Try to acquire lock with sequence number 3. Should update the @@ -157,17 +147,15 @@ start-waiting: false acquire r=req5 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4] -local: num=0 dequeue r=req5 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4] -local: num=0 # ------------------------------------------------------------- # Acquire lock with sequence numbers 5 @@ -185,14 +173,12 @@ start-waiting: false acquire r=req6 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4, 5] -local: num=0 dequeue r=req6 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [1, 2, 3, 4, 5] -local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered index f80389d8b475..bdaa2f6f9da9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered @@ -40,17 +40,15 @@ start-waiting: false acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req2 txn=txn2 ts=10,1 spans=w@a ---- @@ -68,13 +66,12 @@ start-waiting: true release txn=txn1 span=a ---- -global: num=1 +num=1 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, seq: 0 queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 -local: num=0 guard-state r=req2 ---- @@ -90,12 +87,11 @@ new: state=waitForDistinguished txn=txn2 key="a" held=false guard-access=write add-discovered r=req2 k=a txn=txn3 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 -local: num=0 scan r=req2 ---- @@ -111,20 +107,18 @@ new: state=doneWaiting dequeue r=req3 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 -local: num=0 release txn=txn3 span=a ---- -global: num=1 +num=1 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, seq: 0 -local: num=0 guard-state r=req2 ---- @@ -132,5 +126,4 @@ new: state=doneWaiting dequeue r=req2 ---- -global: num=0 -local: num=0 +num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease index 441408872c81..bffa1d5fb515 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered_old_lease @@ -12,8 +12,7 @@ new-request r=req1 txn=txn1 ts=10 spans=w@a+r@b+w@c clear disable ---- -global: num=0 -local: num=0 +num=0 enable lease-seq=5 ---- @@ -24,15 +23,13 @@ start-waiting: false add-discovered r=req1 k=a txn=txn2 lease-seq=4 ---- -global: num=0 -local: num=0 +num=0 add-discovered r=req1 k=b txn=txn2 lease-seq=5 ---- -global: num=1 +num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 add-discovered r=req1 k=c txn=txn2 lease-seq=6 ---- @@ -40,7 +37,6 @@ unexpected lease sequence: 6 > 5 print ---- -global: num=1 +num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic index 2898f651aec9..58aa18999e96 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic @@ -21,33 +21,29 @@ new: state=doneWaiting # uncontended replicated locks are not tracked by lockTable. acquire r=req1 k=c durability=r ---- -global: num=0 -local: num=0 +num=0 acquire r=req1 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=e durability=u ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # 200ms passes between req1 and req2 time-tick ms=200 @@ -64,25 +60,23 @@ start-waiting: false acquire r=req2 k=b durability=u ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req2 ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # 1s passes before txn2 begins time-tick s=1 @@ -104,14 +98,13 @@ start-waiting: false dequeue r=req3 ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # 200ms passes between req3 and req4 time-tick ms=200 @@ -140,14 +133,13 @@ time-tick s=3 # Release lock on b since epoch of txn1 has changed. update txn=txn1 ts=11,1 epoch=1 span=b ---- -global: num=3 +num=3 lock: "b" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # Still waiting, but on lock c which has a different ts in the TxnMeta. @@ -162,14 +154,13 @@ time-tick s=1 # Release lock on c since epoch of txn1 has changed. update txn=txn1 ts=11,1 epoch=1 span=c,e ---- -global: num=3 +num=3 lock: "b" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "c" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # No longer waiting since does not conflict with lock on e. @@ -191,7 +182,7 @@ new-txn txn=txn3 ts=6 epoch=0 add-discovered r=req4 k=a txn=txn3 ---- -global: num=4 +num=4 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -202,11 +193,10 @@ global: num=4 res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 add-discovered r=req4 k=f txn=txn3 ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -219,7 +209,6 @@ global: num=5 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 # Note that guard state has not changed yet. Discovering these locks means the caller has to # scan again. @@ -256,7 +245,7 @@ new: state=waitForDistinguished txn=txn3 key="a" held=true guard-access=write print ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -270,7 +259,6 @@ global: num=5 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 query ---- @@ -389,7 +377,7 @@ start-waiting: false dequeue r=req5 ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -403,7 +391,6 @@ global: num=5 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 # 100ms passes between req5 and req6 time-tick ms=100 @@ -436,7 +423,7 @@ start-waiting: true print ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -453,7 +440,6 @@ global: num=5 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 metrics ---- @@ -589,7 +575,7 @@ new: state=waitForDistinguished txn=txn2 key="c" held=false guard-access=write print ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -609,7 +595,6 @@ global: num=5 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 metrics ---- @@ -732,7 +717,7 @@ time-tick ms=100 # Release a. req4 waits at f. release txn=txn3 span=a ---- -global: num=5 +num=5 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" @@ -749,7 +734,6 @@ global: num=5 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 guard-state r=req4 ---- @@ -761,7 +745,7 @@ old: state=waitForDistinguished txn=txn2 key="b" held=false guard-access=write print ---- -global: num=5 +num=5 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" @@ -781,7 +765,6 @@ global: num=5 waiting readers: req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 -local: num=0 metrics ---- @@ -887,7 +870,7 @@ time-tick ms=500 release txn=txn3 span=f ---- -global: num=4 +num=4 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" @@ -902,7 +885,6 @@ global: num=4 distinguished req: 7 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req4 ---- @@ -930,7 +912,7 @@ start-waiting: false acquire r=req4 k=b durability=r ---- -global: num=4 +num=4 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" @@ -945,11 +927,10 @@ global: num=4 distinguished req: 7 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req4 k=c durability=r ---- -global: num=4 +num=4 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" @@ -964,7 +945,6 @@ global: num=4 distinguished req: 7 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req6 ---- @@ -976,7 +956,7 @@ new: state=waitForDistinguished txn=txn2 key="c" held=true guard-access=write print ---- -global: num=4 +num=4 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 lock: "b" @@ -991,7 +971,6 @@ global: num=4 distinguished req: 7 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 metrics ---- @@ -1089,7 +1068,7 @@ topklocksbywaitduration: dequeue r=req4 ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: @@ -1102,7 +1081,6 @@ global: num=3 distinguished req: 7 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # Locks: # a b c d e f g @@ -1126,7 +1104,7 @@ time-tick s=2 # req7 will get the reservation at c and will become doneWaiting. release txn=txn2 span=c,f ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: @@ -1136,7 +1114,6 @@ global: num=3 res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req7 ---- @@ -1148,7 +1125,7 @@ old: state=waitForDistinguished txn=txn2 key="b" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] queued writers: @@ -1158,7 +1135,6 @@ global: num=3 res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 metrics ---- @@ -1261,14 +1237,13 @@ time-tick ms=40 release txn=txn2 span=b ---- -global: num=3 +num=3 lock: "b" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 lock: "c" res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req6 ---- @@ -1296,7 +1271,7 @@ old: state=doneWaiting print ---- -global: num=3 +num=3 lock: "b" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, seq: 0 lock: "c" @@ -1305,7 +1280,6 @@ global: num=3 active: false req: 7, txn: 00000000-0000-0000-0000-000000000003 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 metrics ---- @@ -1412,12 +1386,11 @@ start-waiting: false # Release reservation. dequeue r=req6 ---- -global: num=2 +num=2 lock: "c" res: req: 7, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req7 ---- @@ -1429,10 +1402,9 @@ start-waiting: false dequeue r=req7 ---- -global: num=1 +num=1 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # e is still locked @@ -1454,10 +1426,9 @@ new: state=waitForDistinguished txn=txn1 key="e" held=true guard-access=write dequeue r=req8 ---- -global: num=1 +num=1 lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # 100ms passes between before releasing c-f time-tick ms=100 @@ -1466,13 +1437,11 @@ time-tick ms=100 release txn=txn1 span=c,f ---- -global: num=0 -local: num=0 +num=0 print ---- -global: num=0 -local: num=0 +num=0 metrics ---- @@ -1573,24 +1542,21 @@ start-waiting: false acquire r=req9 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] -local: num=0 dequeue r=req9 ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] -local: num=0 print ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] -local: num=0 metrics ---- @@ -1715,7 +1681,7 @@ new: state=waitFor txn=txn1 key="c" held=true guard-access=write print ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 1, seqs: [0] queued writers: @@ -1723,7 +1689,6 @@ global: num=1 active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 10 -local: num=0 metrics ---- @@ -1817,14 +1782,13 @@ topklocksbywaitduration: release txn=txn1 span=c ---- -global: num=1 +num=1 lock: "c" res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 -local: num=0 guard-state r=req10 ---- @@ -1844,14 +1808,13 @@ start-waiting: false print ---- -global: num=1 +num=1 lock: "c" res: req: 10, txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, seq: 0 queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 active: true req: 12, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 11 -local: num=0 metrics ---- @@ -1944,13 +1907,12 @@ topklocksbywaitduration: acquire r=req10 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 -local: num=0 guard-state r=req11 ---- @@ -1963,13 +1925,12 @@ new: state=doneWaiting print ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 -local: num=0 metrics ---- @@ -2063,33 +2024,30 @@ topklocksbywaitduration: dequeue r=req10 ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 -local: num=0 acquire r=req12 k=c durability=r ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 -local: num=0 dequeue r=req12 ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 -local: num=0 guard-state r=req11 ---- @@ -2097,10 +2055,9 @@ old: state=waitForDistinguished txn=txn2 key="c" held=true guard-access=write release txn=txn2 span=b,d ---- -global: num=1 +num=1 lock: "c" res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 -local: num=0 guard-state r=req11 ---- @@ -2108,10 +2065,9 @@ new: state=doneWaiting print ---- -global: num=1 +num=1 lock: "c" res: req: 11, txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, seq: 0 -local: num=0 metrics ---- @@ -2202,8 +2158,7 @@ topklocksbywaitduration: dequeue r=req11 ---- -global: num=0 -local: num=0 +num=0 # Tests with non-transactional requests that triggered nil pointer # dereference bugs. @@ -2217,10 +2172,9 @@ start-waiting: false acquire r=req13 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req14 txn=txn1 ts=9,0 spans=w@c ---- @@ -2238,17 +2192,15 @@ start-waiting: true release txn=txn2 span=b,d ---- -global: num=1 +num=1 lock: "c" res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, seq: 0 -local: num=0 dequeue r=req15 ---- -global: num=1 +num=1 lock: "c" res: req: 14, txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, seq: 0 -local: num=0 new-request r=req16 txn=none ts=10,12 spans=r@c ---- @@ -2259,18 +2211,15 @@ start-waiting: false dequeue r=req14 ---- -global: num=0 -local: num=0 +num=0 dequeue r=req16 ---- -global: num=0 -local: num=0 +num=0 print ---- -global: num=0 -local: num=0 +num=0 metrics ---- @@ -2372,28 +2321,25 @@ start-waiting: false acquire r=req17 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 acquire r=req17 k=d durability=u ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 dequeue r=req17 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 new-request r=req18 txn=txn2 ts=10,0 spans=w@c+w@d ---- @@ -2411,7 +2357,7 @@ start-waiting: true print ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 9.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: @@ -2422,7 +2368,6 @@ global: num=2 queued writers: active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 -local: num=0 metrics ---- @@ -2519,7 +2464,7 @@ topklocksbywaitduration: release txn=txn1 span=c ---- -global: num=2 +num=2 lock: "c" res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" @@ -2527,7 +2472,6 @@ global: num=2 queued writers: active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 -local: num=0 guard-state r=req18 ---- @@ -2535,7 +2479,7 @@ new: state=waitFor txn=txn1 key="d" held=true guard-access=write print ---- -global: num=2 +num=2 lock: "c" res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" @@ -2544,7 +2488,6 @@ global: num=2 active: true req: 18, txn: 00000000-0000-0000-0000-000000000002 active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 19 -local: num=0 metrics ---- @@ -2638,14 +2581,13 @@ topklocksbywaitduration: release txn=txn1 span=d ---- -global: num=2 +num=2 lock: "c" res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 19, txn: 00000000-0000-0000-0000-000000000002 -local: num=0 scan r=req18 ---- @@ -2653,12 +2595,11 @@ start-waiting: false acquire r=req18 k=d durability=u ---- -global: num=2 +num=2 lock: "c" res: req: 18, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req19 ---- @@ -2670,22 +2611,19 @@ start-waiting: false dequeue r=req18 ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req19 ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 release txn=txn2 span=d ---- -global: num=0 -local: num=0 +num=0 # Reservation can be broken while holding latches because a different # lock is released @@ -2699,17 +2637,15 @@ start-waiting: false acquire r=req20 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 dequeue r=req20 ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 new-request r=req21 txn=txn1 ts=10 spans=w@d ---- @@ -2720,21 +2656,19 @@ start-waiting: false acquire r=req21 k=d durability=u ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 dequeue r=req21 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 new-request r=req22 txn=txn2 ts=10 spans=w@c+w@d ---- @@ -2745,7 +2679,7 @@ start-waiting: true print ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: @@ -2753,7 +2687,6 @@ global: num=2 distinguished req: 22 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] -local: num=0 metrics ---- @@ -2855,7 +2788,7 @@ start-waiting: true release txn=txn1 span=d ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [0] queued writers: @@ -2863,7 +2796,6 @@ global: num=2 distinguished req: 22 lock: "d" res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req23 ---- @@ -2880,12 +2812,11 @@ start-waiting: false release txn=txn1 span=c ---- -global: num=2 +num=2 lock: "c" res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" res: req: 23, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req22 ---- @@ -2893,14 +2824,13 @@ new: state=doneWaiting print ---- -global: num=2 +num=2 lock: "c" res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: false req: 23, txn: 00000000-0000-0000-0000-000000000003 -local: num=0 metrics ---- @@ -2993,38 +2923,33 @@ topklocksbywaitduration: acquire r=req23 k=d durability=u ---- -global: num=2 +num=2 lock: "c" res: req: 22, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: false req: 22, txn: 00000000-0000-0000-0000-000000000002 -local: num=0 dequeue r=req22 ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req23 ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 release txn=txn3 span=d ---- -global: num=0 -local: num=0 +num=0 print ---- -global: num=0 -local: num=0 +num=0 metrics ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear index d73aea9107c9..429b5fe37407 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear @@ -27,28 +27,25 @@ new: state=doneWaiting acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=b durability=u ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # In its next request, txn1 discovers a lock at c held by txn2. @@ -65,14 +62,13 @@ new: state=doneWaiting add-discovered r=req2 k=c txn=txn2 ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 # A non-transactional read comes in at a and blocks on the lock. @@ -115,7 +111,7 @@ new: state=waitForDistinguished txn=txn1 key="b" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -130,14 +126,12 @@ global: num=3 distinguished req: 5 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,1, info: repl epoch: 0, seqs: [0] -local: num=0 # Clearing removes all locks and allows all waiting requests to proceed. clear ---- -global: num=0 -local: num=0 +num=0 guard-state r=req2 ---- @@ -157,8 +151,7 @@ start-waiting: false dequeue r=req3 ---- -global: num=0 -local: num=0 +num=0 guard-state r=req4 ---- @@ -170,8 +163,7 @@ start-waiting: false dequeue r=req4 ---- -global: num=0 -local: num=0 +num=0 guard-state r=req5 ---- @@ -183,5 +175,4 @@ start-waiting: false dequeue r=req5 ---- -global: num=0 -local: num=0 +num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks index 348e7bfd3879..8adc8b4df928 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks @@ -33,16 +33,15 @@ start-waiting: false add-discovered r=req1 k=a txn=txn2 ---- -global: num=1 +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 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -51,11 +50,10 @@ global: num=2 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 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -68,11 +66,10 @@ global: num=3 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 +num=4 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -89,7 +86,6 @@ global: num=4 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 ---- @@ -97,7 +93,7 @@ start-waiting: false acquire r=req2 k=c durability=u ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -116,11 +112,10 @@ global: num=5 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 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -139,7 +134,6 @@ global: num=5 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 ---- @@ -147,7 +141,7 @@ start-waiting: true print ---- -global: num=5 +num=5 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -167,7 +161,6 @@ global: num=5 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 ---- @@ -182,7 +175,7 @@ new: state=waitForDistinguished txn=txn4 key="c" held=true guard-access=write print ---- -global: num=5 +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: @@ -205,14 +198,13 @@ global: num=5 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 +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: @@ -232,7 +224,6 @@ global: num=5 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 @@ -246,7 +237,7 @@ Intents to resolve: print ---- -global: num=5 +num=5 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 lock: "b" @@ -257,17 +248,14 @@ global: num=5 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 +num=0 clear ---- -global: num=0 -local: num=0 +num=0 # ----------------------------------------------------------------------------- # req3 waits for replicated and unreplicated locks held by txn2. When txn2 is @@ -287,16 +275,15 @@ start-waiting: false add-discovered r=req3 k=a txn=txn2 ---- -global: num=1 +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 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -305,7 +292,6 @@ global: num=2 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 ---- @@ -313,7 +299,7 @@ start-waiting: false acquire r=req4 k=b durability=u ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -324,11 +310,10 @@ global: num=3 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 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -339,7 +324,6 @@ global: num=3 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 ---- @@ -357,22 +341,19 @@ Intents to resolve: print ---- -global: num=2 +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 +num=0 clear ---- -global: num=0 -local: num=0 +num=0 # ----------------------------------------------------------------------------- # req5 waits for replicated locks held by txn2. When txn2 is @@ -392,16 +373,15 @@ start-waiting: false add-discovered r=req5 k=a txn=txn2 ---- -global: num=1 +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 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -410,7 +390,6 @@ global: num=2 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 ---- @@ -418,7 +397,7 @@ start-waiting: true print ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -428,7 +407,6 @@ global: num=2 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 ---- @@ -447,7 +425,7 @@ new: state=waitForDistinguished txn=txn2 key="a" held=true guard-access=write print ---- -global: num=2 +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: @@ -458,11 +436,10 @@ global: num=2 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 +num=2 lock: "a" res: req: 5, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 queued writers: @@ -472,7 +449,6 @@ global: num=2 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 ---- @@ -486,7 +462,7 @@ Intents to resolve: print ---- -global: num=2 +num=2 lock: "a" res: req: 5, txn: 00000000-0000-0000-0000-000000000001, ts: 12.000000000,1, seq: 0 queued writers: @@ -494,24 +470,20 @@ global: num=2 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 +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 +num=0 clear ---- -global: num=0 -local: num=0 +num=0 # ----------------------------------------------------------------------------- # req7 waits for replicated locks held by txn2. When txn2 is finalized, it is @@ -531,16 +503,15 @@ start-waiting: false add-discovered r=req7 k=a txn=txn2 ---- -global: num=1 +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 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -549,7 +520,6 @@ global: num=2 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 ---- @@ -561,7 +531,7 @@ new: state=waitForDistinguished txn=txn2 key="a" held=true guard-access=write print ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] queued writers: @@ -571,7 +541,6 @@ global: num=2 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 ---- @@ -593,27 +562,23 @@ new: state=doneWaiting print ---- -global: num=2 +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 +num=0 dequeue r=req8 ---- -global: num=0 -local: num=0 +num=0 clear ---- -global: num=0 -local: num=0 +num=0 # ----------------------------------------------------------------------------- # Replicated locks are held by txn3, txn4. When txn3 is finalized, req9 notices @@ -633,34 +598,31 @@ start-waiting: false add-discovered r=req9 k=a txn=txn3 ---- -global: num=1 +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 +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 +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 +num=4 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] lock: "b" @@ -669,7 +631,6 @@ global: num=4 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 ---- @@ -684,7 +645,7 @@ new: state=waitForDistinguished txn=txn4 key="c" held=true guard-access=read print ---- -global: num=4 +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" @@ -696,7 +657,6 @@ global: num=4 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 ---- @@ -714,10 +674,9 @@ txn-finalized txn=txn4 status=aborted release txn=txn4 span=c ---- -global: num=1 +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 ---- @@ -727,8 +686,7 @@ Intents to resolve: print ---- -global: num=0 -local: num=0 +num=0 # ----------------------------------------------------------------------------- # req11 is a non-transactional request that finds a lock from a finalized txn @@ -746,12 +704,11 @@ start-waiting: false add-discovered r=req11 k=a txn=txn2 ---- -global: num=1 +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 ---- @@ -762,8 +719,7 @@ start-waiting: true print ---- -global: num=0 -local: num=0 +num=0 guard-state r=req11 ---- @@ -773,13 +729,11 @@ Intents to resolve: dequeue r=req11 ---- -global: num=0 -local: num=0 +num=0 clear ---- -global: num=0 -local: num=0 +num=0 # ----------------------------------------------------------------------------- # req12 is a read request that finds a lock from a finalized txn @@ -795,10 +749,9 @@ start-waiting: false add-discovered r=req12 k=a txn=txn2 ---- -global: num=1 +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 ---- @@ -809,8 +762,7 @@ start-waiting: true print ---- -global: num=0 -local: num=0 +num=0 guard-state r=req12 ---- @@ -820,10 +772,8 @@ Intents to resolve: dequeue r=req12 ---- -global: num=0 -local: num=0 +num=0 clear ---- -global: num=0 -local: num=0 +num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable index 6aeb71adfe7f..c845ca8f79b8 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable @@ -17,8 +17,7 @@ new-txn txn=txn2 ts=10,1 epoch=0 clear disable ---- -global: num=0 -local: num=0 +num=0 new-request r=req1 txn=txn1 ts=10,1 spans=w@a+w@c ---- @@ -33,8 +32,7 @@ new: state=doneWaiting add-discovered r=req1 k=a txn=txn2 ---- -global: num=0 -local: num=0 +num=0 # NOTE: this won't end up in an infinite loop of scanning a disabled # lock-table and discovering but ignoring the same lock in practice @@ -47,13 +45,11 @@ start-waiting: false acquire r=req1 k=c durability=u ---- -global: num=0 -local: num=0 +num=0 dequeue r=req1 ---- -global: num=0 -local: num=0 +num=0 # ------------------------------------------------------------- # Enable the lock-table - the behavior should return to normal. @@ -75,12 +71,11 @@ new: state=doneWaiting add-discovered r=req2 k=a txn=txn2 ---- -global: num=1 +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: 2, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 scan r=req2 ---- @@ -92,10 +87,9 @@ new: state=waitForDistinguished txn=txn2 key="a" held=true guard-access=write release txn=txn2 span=a ---- -global: num=1 +num=1 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 -local: num=0 guard-state r=req2 ---- @@ -103,16 +97,14 @@ new: state=doneWaiting acquire r=req2 k=c durability=u ---- -global: num=2 +num=2 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req2 ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache b/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache index 5ec934abcca8..82e52e19c0c5 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/discovered_locks_consults_txn_cache @@ -27,12 +27,11 @@ txn-finalized txn=txn2 status=aborted # Don't consult finalizedTxnCache. add-discovered r=req1 k=a txn=txn2 consult-finalized-txn-cache=false ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 # Nothing to resolve yet. resolve-before-scanning r=req1 @@ -51,10 +50,9 @@ Intents to resolve: print ---- -global: num=1 +num=1 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 -local: num=0 scan r=req1 ---- @@ -66,30 +64,27 @@ txn-finalized txn=txn3 status=aborted # Txn is finalized and finalizedTxnCache is consulted. add-discovered r=req1 k=b txn=txn3 consult-finalized-txn-cache=true ---- -global: num=1 +num=1 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 -local: num=0 # Txn is finalized and finalizedTxnCache is consulted. add-discovered r=req1 k=c txn=txn3 consult-finalized-txn-cache=true ---- -global: num=1 +num=1 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 -local: num=0 # Txn is not finalized and finalizedTxnCache is consulted. add-discovered r=req1 k=d txn=txn4 consult-finalized-txn-cache=true ---- -global: num=2 +num=2 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000004, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 # Locks for b and c were not added to lock table. resolve-before-scanning r=req1 @@ -108,7 +103,6 @@ new: state=waitForDistinguished txn=txn4 key="d" held=true guard-access=write dequeue r=req1 ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000004, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] -local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access index ee690451db9f..fcc5dbc8d359 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access @@ -26,17 +26,15 @@ start-waiting: false acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req2 txn=txn2 ts=10 spans=w@a+r@a ---- @@ -51,20 +49,18 @@ new: state=waitForDistinguished txn=txn1 key="a" held=true guard-access=write print ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 -local: num=0 release txn=txn1 span=a ---- -global: num=1 +num=1 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req2 ---- @@ -72,8 +68,7 @@ new: state=doneWaiting dequeue r=req2 ---- -global: num=0 -local: num=0 +num=0 # --------------------------------------------------------------------------------- # Test: req5 accesses "b" as both write and read. It has its reservation at "b" @@ -89,41 +84,37 @@ start-waiting: false acquire r=req3 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req3 k=b durability=u ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req3 k=c durability=u ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req3 ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req4 txn=txn2 ts=10 spans=w@a+w@b ---- @@ -149,7 +140,7 @@ new: state=waitForDistinguished txn=txn1 key="b" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -162,13 +153,12 @@ global: num=3 distinguished req: 5 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 # req5 reserves "b" and waits at "c". release txn=txn1 span=b ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -178,7 +168,6 @@ global: num=3 res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req5 ---- @@ -186,7 +175,7 @@ new: state=waitForDistinguished txn=txn1 key="c" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -199,13 +188,12 @@ global: num=3 queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 -local: num=0 # req4 breaks the reservation of req4 at "b". release txn=txn1 span=a ---- -global: num=3 +num=3 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -215,7 +203,6 @@ global: num=3 queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 -local: num=0 guard-state r=req4 ---- @@ -223,7 +210,7 @@ new: state=doneWaiting print ---- -global: num=3 +num=3 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -235,13 +222,12 @@ global: num=3 queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 -local: num=0 # req5 encounters the reservation by req4 at "b" when looking at it for its read access, but ignores # it. release txn=txn1 span=c ---- -global: num=3 +num=3 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -250,7 +236,6 @@ global: num=3 active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req5 ---- @@ -262,7 +247,7 @@ start-waiting: true print ---- -global: num=3 +num=3 lock: "a" res: req: 4, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -272,26 +257,22 @@ global: num=3 distinguished req: 5 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 dequeue r=req4 ---- -global: num=2 +num=2 lock: "b" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 dequeue r=req5 ---- -global: num=0 -local: num=0 +num=0 print ---- -global: num=0 -local: num=0 +num=0 # --------------------------------------------------------------------------------- # Test: Non-transactional req8 accesses "b" as both write and read. After it has stopped waiting @@ -309,41 +290,37 @@ start-waiting: false acquire r=req6 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req6 k=b durability=u ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req6 k=c durability=u ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req6 ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req7 txn=txn2 ts=10 spans=w@a+w@b ---- @@ -383,7 +360,7 @@ new: state=waitFor txn=txn1 key="b" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -397,13 +374,12 @@ global: num=3 distinguished req: 8 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 # req8 waits at "c". release txn=txn1 span=b ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -413,7 +389,6 @@ global: num=3 res: req: 9, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req8 ---- @@ -421,7 +396,7 @@ new: state=waitForDistinguished txn=txn1 key="c" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -434,13 +409,12 @@ global: num=3 queued writers: active: true req: 8, txn: none distinguished req: 8 -local: num=0 # req7 is doneWaiting and proceeds to acquire the lock at "b". release txn=txn1 span=a ---- -global: num=3 +num=3 lock: "a" res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -450,7 +424,6 @@ global: num=3 queued writers: active: true req: 8, txn: none distinguished req: 8 -local: num=0 guard-state r=req7 ---- @@ -458,7 +431,7 @@ new: state=doneWaiting print ---- -global: num=3 +num=3 lock: "a" res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -470,7 +443,6 @@ global: num=3 queued writers: active: true req: 8, txn: none distinguished req: 8 -local: num=0 scan r=req7 ---- @@ -478,7 +450,7 @@ start-waiting: false acquire r=req7 k=b durability=u ---- -global: num=3 +num=3 lock: "a" res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -488,17 +460,15 @@ global: num=3 queued writers: active: true req: 8, txn: none distinguished req: 8 -local: num=0 # req8 encounters the lock held by req7 at "b" when looking at it for its read access. release txn=txn1 span=c ---- -global: num=2 +num=2 lock: "a" res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req8 ---- @@ -506,7 +476,7 @@ new: state=waitForDistinguished txn=txn2 key="b" held=true guard-access=read print ---- -global: num=2 +num=2 lock: "a" res: req: 7, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" @@ -514,29 +484,25 @@ global: num=2 waiting readers: req: 8, txn: none distinguished req: 8 -local: num=0 dequeue r=req7 ---- -global: num=1 +num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] waiting readers: req: 8, txn: none distinguished req: 8 -local: num=0 dequeue r=req8 ---- -global: num=1 +num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 release txn=txn2 span=b ---- -global: num=0 -local: num=0 +num=0 # --------------------------------------------------------------------------------- # Test: req12 accesses "b" as both write and read. It has its reservation at "b" @@ -553,28 +519,25 @@ start-waiting: false acquire r=req10 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req10 k=b durability=u ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req10 ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req11 txn=txn2 ts=10 spans=w@a+w@b ---- @@ -600,7 +563,7 @@ new: state=waitForDistinguished txn=txn1 key="b" held=true guard-access=write print ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -611,13 +574,12 @@ global: num=2 queued writers: active: true req: 12, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 12 -local: num=0 # req12 reserves "b". release txn=txn1 span=b ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -625,17 +587,15 @@ global: num=2 distinguished req: 11 lock: "b" res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 # req11 reserves "a" release txn=txn1 span=a ---- -global: num=2 +num=2 lock: "a" res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" res: req: 12, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 # req11 breaks the reservation at "b" guard-state r=req11 @@ -644,14 +604,13 @@ new: state=doneWaiting print ---- -global: num=2 +num=2 lock: "a" res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: false req: 12, txn: 00000000-0000-0000-0000-000000000003 -local: num=0 scan r=req11 ---- @@ -659,14 +618,13 @@ start-waiting: false acquire r=req11 k=b durability=u ---- -global: num=2 +num=2 lock: "a" res: req: 11, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: false req: 12, txn: 00000000-0000-0000-0000-000000000003 -local: num=0 # req12 ignores the lock at "b" when it encounters it again as a reader. So it will # enter the doneWaiting state. It will wait again when it rescans. @@ -685,25 +643,22 @@ new: state=waitForDistinguished txn=txn2 key="b" held=true guard-access=write dequeue r=req11 ---- -global: num=1 +num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 12, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 12 -local: num=0 dequeue r=req12 ---- -global: num=1 +num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 release txn=txn2 span=b ----- print ---- -global: num=0 -local: num=0 +num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes index 316971ea888c..5b091edbd325 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes @@ -18,17 +18,15 @@ new-request r=req1 txn=txn1 ts=10 spans=w@a acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] -local: num=0 release txn=txn2 span=a,c ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] -local: num=0 # --------------------------------------------------------------------------------- # Lock is reacquired at same epoch with lower timestamp. This is allowed, @@ -44,10 +42,9 @@ new-request r=req2 txn=txn1 ts=8 spans=w@a acquire r=req2 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2, 3] -local: num=0 # --------------------------------------------------------------------------------- # Lock is reacquired at same epoch with lower timestamp and different durability. @@ -67,20 +64,18 @@ start-waiting: true acquire r=req2 k=a durability=r ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 0, seqs: [2, 3] queued writers: active: true req: 1, txn: none distinguished req: 1 -local: num=0 dequeue r=reqContend ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 0, seqs: [2, 3] -local: num=0 # --------------------------------------------------------------------------------- # Lock is reacquired at a different epoch. The old sequence numbers are discarded. @@ -94,10 +89,9 @@ new-request r=req3 txn=txn1 ts=10 spans=w@a acquire r=req3 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 1, seqs: [0] -local: num=0 # --------------------------------------------------------------------------------- # Lock is reacquired at a different epoch with lower timestamp. This is allowed, @@ -113,10 +107,9 @@ new-request r=req4 txn=txn1 ts=6 spans=w@a acquire r=req4 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 2, seqs: [0] -local: num=0 # --------------------------------------------------------------------------------- # Reader waits until the timestamp of the lock is updated. @@ -135,13 +128,12 @@ new: state=waitForDistinguished txn=txn1 key="a" held=true guard-access=read print ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, seqs: [3], unrepl epoch: 2, seqs: [0] waiting readers: req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 -local: num=0 new-txn txn=txn1 ts=14 epoch=1 seq=1 ---- @@ -151,13 +143,12 @@ new-request r=req6 txn=txn1 ts=14 spans=w@a acquire r=req6 k=a durability=r ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 2, seqs: [0] waiting readers: req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 -local: num=0 guard-state r=req5 ---- @@ -165,10 +156,9 @@ old: state=waitForDistinguished txn=txn1 key="a" held=true guard-access=read acquire r=req6 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 1, seqs: [0, 1] -local: num=0 guard-state r=req5 ---- @@ -194,13 +184,12 @@ new: state=waitForDistinguished txn=txn1 key="a" held=true guard-access=read add-discovered r=req7 k=a txn=txn1 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 1, seqs: [1], unrepl epoch: 1, seqs: [0, 1] waiting readers: req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 -local: num=0 guard-state r=req7 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped index d8d5083f0af5..c99f2ced3d1e 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped @@ -15,8 +15,7 @@ new-request r=req1 txn=txn1 ts=10 spans=w@a acquire r=req1 k=a durability=r ---- -global: num=0 -local: num=0 +num=0 # --------------------------------------------------------------------------------- # Upgrading from unreplicated to replicated for an uncontended lock causes that @@ -25,15 +24,13 @@ local: num=0 acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] -local: num=0 acquire r=req1 k=a durability=r ---- -global: num=0 -local: num=0 +num=0 # --------------------------------------------------------------------------------- # Upgrading from unreplicated to replicated for a lock with only waiting readers @@ -45,10 +42,9 @@ new-request r=reqContendReader txn=none ts=10 spans=r@a acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] -local: num=0 scan r=reqContendReader ---- @@ -56,18 +52,16 @@ start-waiting: true print ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] waiting readers: req: 1, txn: none distinguished req: 1 -local: num=0 acquire r=req1 k=a durability=r ---- -global: num=0 -local: num=0 +num=0 guard-state r=reqContendReader ---- @@ -83,10 +77,9 @@ new-request r=reqContendWriter txn=none ts=10 spans=w@a acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] -local: num=0 scan r=reqContendReader ---- @@ -98,7 +91,7 @@ start-waiting: true print ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] waiting readers: @@ -106,11 +99,10 @@ global: num=1 queued writers: active: true req: 2, txn: none distinguished req: 1 -local: num=0 acquire r=req1 k=a durability=r ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [2], unrepl epoch: 0, seqs: [2] waiting readers: @@ -118,7 +110,6 @@ global: num=1 queued writers: active: true req: 2, txn: none distinguished req: 1 -local: num=0 guard-state r=reqContendReader ---- @@ -130,8 +121,7 @@ new: state=waitFor txn=txn1 key="a" held=true guard-access=write clear ---- -global: num=0 -local: num=0 +num=0 # --------------------------------------------------------------------------------- # Upgrading from unreplicated to replicated for an uncontended lock. The lockState @@ -147,10 +137,9 @@ local: num=0 acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] -local: num=0 new-txn txn=txn2 ts=10 epoch=0 seq=0 ---- @@ -160,12 +149,11 @@ new-request r=req2 txn=txn2 ts=10 spans=w@b acquire r=req2 k=b durability=u ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req3 txn=none ts=10 spans=r@a,c ---- @@ -180,18 +168,16 @@ new: state=waitForDistinguished txn=txn1 key="a" held=true guard-access=read acquire r=req2 k=b durability=r ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [2] waiting readers: req: 3, txn: none distinguished req: 3 -local: num=0 release txn=txn1 span=a ---- -global: num=0 -local: num=0 +num=0 # Before the fix in #50173, this used to enter the following state: # new: state=waitForDistinguished txn=txn2 key="b" held=true guard-access=read diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter index 744a56fe9e86..e360f767962a 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter @@ -18,27 +18,25 @@ start-waiting: false add-discovered r=req1 k=a txn=txn2 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, 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 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, 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,0, info: repl epoch: 0, seqs: [0] -local: num=0 add-discovered r=req1 k=c txn=txn2 ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -49,13 +47,12 @@ global: num=3 holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 # req1 is not in the queue for "b" as readers are never inactive waiters. print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -66,7 +63,6 @@ global: num=3 holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 new-request r=req2 txn=txn1 ts=10 spans=w@c ---- @@ -79,7 +75,7 @@ start-waiting: true print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -92,7 +88,6 @@ global: num=3 active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 -local: num=0 scan r=req1 ---- @@ -104,7 +99,7 @@ new: state=waitForDistinguished txn=txn2 key="a" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: @@ -118,12 +113,11 @@ global: num=3 active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 -local: num=0 # req1 waits at "c" but not as distinguished waiter. release txn=txn2 span=a ---- -global: num=3 +num=3 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" @@ -134,7 +128,6 @@ global: num=3 active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 -local: num=0 guard-state r=req1 ---- @@ -142,7 +135,7 @@ new: state=waitFor txn=txn2 key="c" held=true guard-access=write print ---- -global: num=3 +num=3 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" @@ -153,13 +146,12 @@ global: num=3 active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 2 -local: num=0 # req1 waits at "b" as reader. release txn=txn2 span=c ---- -global: num=3 +num=3 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" @@ -168,7 +160,6 @@ global: num=3 res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 guard-state r=req1 ---- @@ -180,7 +171,7 @@ new: state=waitSelf print ---- -global: num=3 +num=3 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" @@ -192,20 +183,18 @@ global: num=3 res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 # req1 is done waiting. release txn=txn2 span=b ---- -global: num=2 +num=2 lock: "a" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 1, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 -local: num=0 guard-state r=req1 ---- @@ -213,10 +202,9 @@ new: state=doneWaiting dequeue r=req1 ---- -global: num=1 +num=1 lock: "c" res: req: 2, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req2 ---- @@ -224,5 +212,4 @@ new: state=doneWaiting dequeue r=req2 ---- -global: num=0 -local: num=0 +num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write index 338da397d80c..fa7fe0620cd9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write @@ -20,41 +20,37 @@ start-waiting: false acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=b durability=u ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=c durability=u ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 # Next, two different transactional requests wait at a and b. new-request r=req2 txn=txn2 ts=10 spans=w@a @@ -91,7 +87,7 @@ start-waiting: true print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -108,14 +104,13 @@ global: num=3 queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 -local: num=0 # The locks at a, b, c are released. The non-transactional request waits behind # the reservation holder at a. release txn=txn1 span=a,d ---- -global: num=3 +num=3 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: @@ -125,7 +120,6 @@ global: num=3 res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req2 ---- @@ -154,7 +148,7 @@ start-waiting: true print ---- -global: num=3 +num=3 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: @@ -165,7 +159,6 @@ global: num=3 res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 # Release the reservation at a. The first waiter is non-transactional so it will not acquire the # reservation. The second waiter will acquire the reservation. The non-transactional request will @@ -173,14 +166,13 @@ local: num=0 dequeue r=req2 ---- -global: num=3 +num=3 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req4 ---- @@ -192,7 +184,7 @@ new: state=doneWaiting print ---- -global: num=3 +num=3 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "b" @@ -202,19 +194,17 @@ global: num=3 distinguished req: 4 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 # Release the reservation at b. The non-transactional waiter will be done at b, and when it gets # to c it will see a reservation holder with a higher sequence num and ignore it. dequeue r=req3 ---- -global: num=2 +num=2 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req4 ---- @@ -226,12 +216,11 @@ old: state=doneWaiting print ---- -global: num=2 +num=2 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 # Non-transactional request scans again and proceeds to evaluation and discovers a lock at c @@ -241,7 +230,7 @@ start-waiting: false add-discovered r=req4 k=c txn=txn2 ---- -global: num=2 +num=2 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" @@ -249,7 +238,6 @@ global: num=2 queued writers: active: false req: 4, txn: none active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 -local: num=0 scan r=req4 ---- @@ -271,12 +259,11 @@ new: state=waitFor txn=txn2 key="c" held=true guard-access=write release txn=txn2 span=c ---- -global: num=2 +num=2 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 guard-state r=req4 ---- @@ -290,21 +277,18 @@ new: state=doneWaiting dequeue r=req4 ---- -global: num=2 +num=2 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 lock: "c" res: req: 5, txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,0, seq: 0 -local: num=0 dequeue r=req5 ---- -global: num=1 +num=1 lock: "a" res: req: 6, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, seq: 0 -local: num=0 dequeue r=req6 ---- -global: num=0 -local: num=0 +num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic b/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic index c2c55e67ece5..6bdf9490342e 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/optimistic @@ -22,28 +22,25 @@ false acquire r=req1 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=g durability=u ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req2 txn=txn2 ts=11,1 spans=r@a,d ---- @@ -58,12 +55,11 @@ true dequeue r=req2 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req3 txn=txn2 ts=11,1 spans=r@a,d+r@f,i ---- @@ -78,12 +74,11 @@ false print ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 check-opt-no-conflicts r=req3 spans=r@a,c ---- @@ -103,12 +98,11 @@ no-conflicts: false dequeue r=req3 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # --------------------------------------------------------------------------------- # Test with a Skip wait policy. Even though the lock table has a conflicting lock, @@ -133,9 +127,8 @@ no-conflicts: true dequeue r=req4 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "g" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/query b/pkg/kv/kvserver/concurrency/testdata/lock_table/query index 3086f9e86bef..40d0198f25f7 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/query +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/query @@ -19,28 +19,25 @@ new: state=doneWaiting acquire r=req1 k=c durability=u ---- -global: num=1 +num=1 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=e durability=u ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=2 +num=2 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 query span=a,d ---- @@ -63,25 +60,23 @@ start-waiting: false acquire r=req2 k=b durability=u ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req2 ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "e" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # make sure query limits work @@ -138,7 +133,7 @@ time-tick ms=200 print ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,2, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -151,7 +146,6 @@ global: num=3 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 4 -local: num=0 query span=a,/Max max-bytes=100 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded b/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded index 8ffa6f876fb4..789a6cdcadcd 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/queue_length_exceeded @@ -25,10 +25,9 @@ start-waiting: false acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 scan r=req2 ---- @@ -40,14 +39,13 @@ start-waiting: true print ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 -local: num=0 # --------------------------------------------------------------------------------- # Read requests do not observe a queue length limit, because they don't wait in the @@ -70,14 +68,13 @@ new: state=waitFor txn=txn1 key="a" held=true guard-access=read dequeue r=req4 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 -local: num=0 # --------------------------------------------------------------------------------- # Write requests with a large enough MaxLockWaitQueueLength do not throw an error. @@ -99,14 +96,13 @@ new: state=waitFor txn=txn1 key="a" held=true guard-access=write dequeue r=req5 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 -local: num=0 # --------------------------------------------------------------------------------- # Write requests with a sufficiently low MaxLockWaitQueueLength throw an error. @@ -128,14 +124,13 @@ new: state=waitQueueMaxLengthExceeded txn=txn1 key="a" held=true guard-access=wr dequeue r=req6 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 -local: num=0 # --------------------------------------------------------------------------------- # Same as previous two cases, but for non-transactional writes. @@ -154,14 +149,13 @@ new: state=waitFor txn=txn1 key="a" held=true guard-access=write dequeue r=req7 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 -local: num=0 new-request r=req8 txn=none ts=10 spans=w@a max-lock-wait-queue-length=2 ---- @@ -176,11 +170,10 @@ new: state=waitQueueMaxLengthExceeded txn=txn1 key="a" held=true guard-access=wr dequeue r=req8 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 2 -local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded index eae6e6256347..14fdacb87dcf 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded @@ -24,19 +24,17 @@ start-waiting: false acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=b durability=u ---- -global: num=2 +num=2 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 # c is first locked as unreplicated and establishes a writer queue # before being locked as replicated. We really only need it replicated @@ -47,14 +45,13 @@ local: num=0 # acquisition and queued writer. acquire r=req1 k=c durability=u ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=reqContend txn=none ts=10 spans=w@c ---- @@ -65,7 +62,7 @@ start-waiting: true acquire r=req1 k=c durability=r ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" @@ -75,29 +72,26 @@ global: num=3 queued writers: active: true req: 2, txn: none distinguished req: 2 -local: num=0 dequeue r=reqContend ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req2 txn=txn2 ts=10 spans=w@a,c ---- @@ -115,7 +109,7 @@ start-waiting: true print ---- -global: num=3 +num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -126,11 +120,10 @@ global: num=3 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] -local: num=0 release txn=txn1 span=a ---- -global: num=3 +num=3 lock: "a" res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: @@ -139,7 +132,6 @@ global: num=3 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] -local: num=0 guard-state r=req2 ---- @@ -151,7 +143,7 @@ new: state=waitSelf print ---- -global: num=3 +num=3 lock: "a" res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: @@ -163,7 +155,6 @@ global: num=3 distinguished req: 3 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req4 txn=txn2 ts=10 spans=r@b ---- @@ -199,7 +190,7 @@ new: state=doneWaiting add-discovered r=req7 k=d txn=txn1 ---- -global: num=4 +num=4 lock: "a" res: req: 3, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: @@ -221,7 +212,6 @@ global: num=4 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 8, txn: 00000000-0000-0000-0000-000000000002 -local: num=0 new-request r=req8 txn=txn2 ts=10 spans=w@e ---- @@ -234,12 +224,11 @@ start-waiting: false # locks except "d" which is the discovered lock with no active waiter. acquire r=req8 k=e durability=u ---- -global: num=1 +num=1 lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] queued writers: active: false req: 8, txn: 00000000-0000-0000-0000-000000000002 -local: num=0 guard-state r=req2 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked index ca92fc11056e..b8248da00fb2 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked @@ -28,28 +28,25 @@ false acquire r=req1 k=b durability=u ---- -global: num=1 +num=1 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req1 k=d durability=u ---- -global: num=2 +num=2 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=2 +num=2 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req2 txn=txn2 ts=9,1 spans=w@c+w@f ---- @@ -64,18 +61,17 @@ false acquire r=req2 k=c durability=u ---- -global: num=3 +num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 9.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "d" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 acquire r=req2 k=f durability=u ---- -global: num=4 +num=4 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" @@ -84,11 +80,10 @@ global: num=4 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 9.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req2 ---- -global: num=4 +num=4 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" @@ -97,7 +92,6 @@ global: num=4 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 9.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 new-request r=req3 txn=txn1 ts=10,1 spans=w@f ---- @@ -112,7 +106,7 @@ true release txn=txn2 span=f ---- -global: num=4 +num=4 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" @@ -121,7 +115,6 @@ global: num=4 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 -local: num=0 # --------------------------------------------------------------------------------- # req4 will scan the lock table with a Skip wait policy. It will not need to wait. @@ -190,7 +183,7 @@ locked: true, holder: dequeue r=req4 ---- -global: num=4 +num=4 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" @@ -199,7 +192,6 @@ global: num=4 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 -local: num=0 # --------------------------------------------------------------------------------- # req5 is the same as req4, except is has a timestamp equal to txn1's to @@ -243,7 +235,7 @@ locked: false dequeue r=req5 ---- -global: num=4 +num=4 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" @@ -252,4 +244,3 @@ global: num=4 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "f" res: req: 3, txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, seq: 0 -local: num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/update b/pkg/kv/kvserver/concurrency/testdata/lock_table/update index b853a9e4e813..bcca16c67f71 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/update +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/update @@ -27,17 +27,15 @@ new: state=doneWaiting acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] -local: num=0 # ------------------------------------------------------------- # Wait on this lock as: @@ -93,7 +91,7 @@ new: state=waitFor txn=txn1 key="a" held=true guard-access=write print ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -103,7 +101,6 @@ global: num=1 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 2 -local: num=0 metrics ---- @@ -201,7 +198,7 @@ topklocksbywaitduration: update txn=txn1 ts=11,1 epoch=0 span=a ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -211,7 +208,6 @@ global: num=1 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 2 -local: num=0 # ------------------------------------------------------------- # Update lock timestamp to 13,1 - the transactional read at @@ -221,7 +217,7 @@ local: num=0 update txn=txn1 ts=13,1 epoch=0 span=a ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 13.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -230,7 +226,6 @@ global: num=1 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 4 -local: num=0 guard-state r=req2 ---- @@ -242,7 +237,7 @@ start-waiting: false dequeue r=req2 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 13.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -251,7 +246,6 @@ global: num=1 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 4 -local: num=0 # ------------------------------------------------------------- # Update lock timestamp to 10,1 - noop since lock is already at @@ -260,7 +254,7 @@ local: num=0 update txn=txn1 ts=10,1 epoch=0 span=a ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 13.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -269,7 +263,6 @@ global: num=1 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 4 -local: num=0 # ------------------------------------------------------------- # Update lock timestamp to 15,1 - nothing moves @@ -277,7 +270,7 @@ local: num=0 update txn=txn1 ts=15,1 epoch=0 span=a ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 15.000000000,1, info: unrepl epoch: 0, seqs: [0] waiting readers: @@ -286,7 +279,6 @@ global: num=1 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 4 -local: num=0 # ------------------------------------------------------------- # Update lock timestamp to 17,1 - the transactional read at @@ -296,14 +288,13 @@ local: num=0 update txn=txn1 ts=17,1 epoch=0 span=a ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 17.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 3 -local: num=0 guard-state r=req4 ---- @@ -315,14 +306,13 @@ start-waiting: false dequeue r=req4 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 17.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 3 -local: num=0 # ------------------------------------------------------------- # Update lock timestamp to 19,1 - nothing moves @@ -330,14 +320,13 @@ local: num=0 update txn=txn1 ts=19,1 epoch=0 span=a ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 19.000000000,1, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 5, txn: none distinguished req: 3 -local: num=0 # ------------------------------------------------------------- # Update lock epoch to 1 - the lock is dropped and the transactional @@ -348,13 +337,12 @@ local: num=0 update txn=txn1 ts=19,1 epoch=1 span=a ---- -global: num=1 +num=1 lock: "a" res: req: 3, txn: 00000000-0000-0000-0000-000000000003, ts: 14.000000000,1, seq: 0 queued writers: active: true req: 5, txn: none distinguished req: 5 -local: num=0 guard-state r=req3 ---- @@ -370,8 +358,7 @@ new: state=waitForDistinguished txn=txn3 key="a" held=false guard-access=write dequeue r=req3 ---- -global: num=0 -local: num=0 +num=0 guard-state r=req5 ---- @@ -383,8 +370,7 @@ start-waiting: false dequeue r=req5 ---- -global: num=0 -local: num=0 +num=0 # ------------------------------------------------------------- # Lock is held at multiple seqnums and then updated to ignore @@ -405,10 +391,9 @@ new-request r=req1 txn=txn1 ts=10 spans=w@a acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1] -local: num=0 new-txn txn=txn1 ts=10 epoch=1 seq=5 ---- @@ -418,10 +403,9 @@ new-request r=req2 txn=txn1 ts=10 spans=w@a acquire r=req2 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5] -local: num=0 new-txn txn=txn1 ts=10 epoch=1 seq=7 ---- @@ -431,10 +415,9 @@ new-request r=req3 txn=txn1 ts=10 spans=w@a acquire r=req3 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7] -local: num=0 new-txn txn=txn1 ts=10 epoch=1 seq=10 ---- @@ -444,74 +427,65 @@ new-request r=req4 txn=txn1 ts=10 spans=w@a acquire r=req4 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] -local: num=0 # No seqnum change since lock is not held at seqnum 3, 8, 9. update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=3,8-9 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] -local: num=0 # No change since update is using older epoch. update txn=txn1 ts=10 epoch=0 span=a ignored-seqs=3,5-7 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 5, 7, 10] -local: num=0 update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=3,5-7 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1, 10] -local: num=0 update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=9-11 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 1, seqs: [1] -local: num=0 # No seqnum change since update is using older epoch. But since the update is using # a higher timestamp, the ts is advanced. update txn=txn1 ts=15 epoch=0 span=a ignored-seqs=1 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 15.000000000,0, info: unrepl epoch: 1, seqs: [1] -local: num=0 # No change, since seqnum 3 is not held. Note that the ts is not updated. update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=3 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 15.000000000,0, info: unrepl epoch: 1, seqs: [1] -local: num=0 # Timestamp is updated again. update txn=txn1 ts=16 epoch=1 span=a ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 16.000000000,0, info: unrepl epoch: 1, seqs: [1] -local: num=0 # Seqnum 1 is also ignored, so the lock is released. Note that it does not # matter that the update is using an older timestamp. update txn=txn1 ts=10 epoch=1 span=a ignored-seqs=1 ---- -global: num=0 -local: num=0 +num=0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self b/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self index 03a76537bbb9..6238a3c437a6 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/wait_self @@ -35,17 +35,15 @@ start-waiting: false acquire r=req1 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 dequeue r=req1 ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] -local: num=0 scan r=req2 ---- @@ -73,7 +71,7 @@ new: state=waitFor txn=txn1 key="a" held=true guard-access=write print ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: @@ -81,18 +79,16 @@ global: num=1 active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 -local: num=0 release txn=txn1 span=a ---- -global: num=1 +num=1 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 -local: num=0 guard-state r=req2 ---- @@ -108,14 +104,13 @@ new: state=waitSelf print ---- -global: num=1 +num=1 lock: "a" res: req: 2, txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, seq: 0 queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 -local: num=0 # Stays in waitSelf state if scans again. scan r=req4 @@ -133,13 +128,12 @@ new: state=waitSelf acquire r=req2 k=a durability=u ---- -global: num=1 +num=1 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: unrepl epoch: 0, seqs: [0] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 3 -local: num=0 guard-state r=req3 ----