From 0147fccbbf7fa1cd84abf56fc4ddefa212272264 Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Tue, 27 Jun 2023 12:04:30 -0400 Subject: [PATCH 1/2] kvserver: deflake TestProtectedTimestamps This patch fixes a few (hopefully all) issues with TestProtectedTimestamps. In particular, - The range max bytes used by the test was broken after the lower bound was bumped in a37e053173ebf069b12ef6a2c38a03dd984992e2. We up the value. - There was flakiness at various points in the test as a result of lease transfers. We change the test to run on a single node test cluster to get around this. Fixes: #93497 Release note: None --- pkg/kv/kvserver/client_protectedts_test.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/client_protectedts_test.go b/pkg/kv/kvserver/client_protectedts_test.go index a76965347732..2d30e2e1be71 100644 --- a/pkg/kv/kvserver/client_protectedts_test.go +++ b/pkg/kv/kvserver/client_protectedts_test.go @@ -47,7 +47,6 @@ import ( func TestProtectedTimestamps(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) - skip.WithIssue(t, 93497, "flaky test") ctx := context.Background() // This test is too slow to run with race. @@ -59,7 +58,7 @@ func TestProtectedTimestamps(t *testing.T) { DisableGCQueue: true, DisableLastProcessedCheck: true, } - tc := testcluster.StartTestCluster(t, 3, args) + tc := testcluster.StartTestCluster(t, 1, args) defer tc.Stopper().Stop(ctx) s0 := tc.Server(0) @@ -73,7 +72,7 @@ func TestProtectedTimestamps(t *testing.T) { _, err = conn.Exec("SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'") // speeds up the test require.NoError(t, err) - const tableRangeMaxBytes = 1 << 18 + const tableRangeMaxBytes = 64 << 20 _, err = conn.Exec("ALTER TABLE foo CONFIGURE ZONE USING "+ "gc.ttlseconds = 1, range_max_bytes = $1, range_min_bytes = 1<<10;", tableRangeMaxBytes) require.NoError(t, err) From ca5a95966b6c81f78f50c728d79238a196dc2e8c Mon Sep 17 00:00:00 2001 From: Arul Ajmani Date: Thu, 22 Jun 2023 15:45:56 -0400 Subject: [PATCH 2/2] concurrency: split out replicated/unreplicated lock holder information The MVCC keyspace is the source of truth for replicated locks. We've previously concluded that trying to keep the in-memory state of replicated locks in sync is fraught with subtle issues. We instead do the dumb thing and forget replicated locks in a few places (see comment about mvccResolveWriteIntent in tryUpdateLockLocked). It follows that we don't need to track as much information about replicated locks (like we do for unreplicated locks). For example, we do not need to track the sequence number history for replicated locks. This patch splits out lock holder information about replicated locks and unreplicated locks into 2 different structs. As mentioned above, we no longer track seqeunce numbers for the former. Informs #102270 Release note: None --- pkg/kv/kvserver/concurrency/lock_table.go | 460 +++++++++++------- .../kvserver/concurrency/lock_table_test.go | 7 +- .../clear_abandoned_intents | 88 ++-- ...doned_intents_without_adding_to_lock_table | 4 +- .../discover_lock_after_lease_race | 4 +- .../concurrency_manager/discovered_lock | 2 +- .../testdata/concurrency_manager/lock_timeout | 4 +- .../concurrency_manager/range_state_listener | 10 +- .../resolve_pushed_intents | 54 +- ...ushed_intents_without_adding_to_lock_table | 42 +- .../testdata/concurrency_manager/uncertainty | 8 +- .../testdata/concurrency_manager/update | 2 +- .../concurrency_manager/wait_elsewhere | 8 +- .../concurrency_manager/wait_policy_error | 4 +- .../testdata/lock_table/acquire_ignored_seqs | 33 +- .../testdata/lock_table/add_discovered | 4 +- .../lock_table/add_discovered_old_lease | 4 +- .../concurrency/testdata/lock_table/basic | 48 +- .../concurrency/testdata/lock_table/clear | 4 +- .../lock_table/clear_finalized_txn_locks | 152 +++--- .../concurrency/testdata/lock_table/disable | 2 +- .../discovered_locks_consults_txn_cache | 12 +- .../testdata/lock_table/lock_changes | 18 +- .../testdata/lock_table/lock_dropped | 2 +- .../testdata/lock_table/non_active_waiter | 42 +- .../testdata/lock_table/non_txn_write | 2 +- .../lock_table/resolve_pushed_txn_locks | 28 +- .../testdata/lock_table/size_limit_exceeded | 18 +- 28 files changed, 583 insertions(+), 483 deletions(-) diff --git a/pkg/kv/kvserver/concurrency/lock_table.go b/pkg/kv/kvserver/concurrency/lock_table.go index 4e46637a03a1..e2afed075b94 100644 --- a/pkg/kv/kvserver/concurrency/lock_table.go +++ b/pkg/kv/kvserver/concurrency/lock_table.go @@ -914,8 +914,8 @@ type queuedGuard struct { active bool // protected by lockState.mu } -// Information about a lock holder. -type lockHolderInfo struct { +// Information about a lock holder for unreplicated locks. +type unreplicatedLockHolderInfo struct { // nil if there is no holder. Else this is the TxnMeta of the latest call to // acquire/update the lock by this transaction. For a given transaction if // the lock is continuously held by a succession of different TxnMetas, the @@ -924,19 +924,92 @@ type lockHolderInfo struct { // is initialized, the timestamps inside txn are not used. txn *enginepb.TxnMeta - // All the TxnSeqs in the current epoch at which this lock has been - // acquired. In increasing order. We track these so that if a lock is - // acquired at both seq 5 and seq 7, rollback of 7 does not cause the lock - // to be released. This is also consistent with PostgreSQL semantics + // All the TxnSeqs in the current epoch at which this lock has been acquired, + // in increasing order. We track these so that if a lock is acquired at both + // seq 5 and seq 7, rollback of 7 does not cause the lock to be released. This + // consistent with PostgreSQL semantics; see: // https://www.postgresql.org/docs/12/sql-select.html#SQL-FOR-UPDATE-SHARE seqs []enginepb.TxnSeq - // The timestamp at which the lock is held. + // The timestamp at which the unreplicated lock is held. ts hlc.Timestamp } -func (lh *lockHolderInfo) isEmpty() bool { - return lh.txn == nil && lh.seqs == nil && lh.ts.IsEmpty() +func (ulh *unreplicatedLockHolderInfo) isEmpty() bool { + return ulh.txn == nil && ulh.seqs == nil && ulh.ts.IsEmpty() +} + +func (ulh *unreplicatedLockHolderInfo) safeFormat( + sb *redact.StringBuilder, txnStatusCache *txnStatusCache, +) { + if ulh.txn == nil { + return + } + sb.SafeString("unrepl ") + if txnStatusCache != nil { + finalizedTxn, ok := txnStatusCache.finalizedTxns.get(ulh.txn.ID) + if ok { + var statusStr string + switch finalizedTxn.Status { + case roachpb.COMMITTED: + statusStr = "committed" + case roachpb.ABORTED: + statusStr = "aborted" + } + sb.Printf("[holder finalized: %s] ", redact.Safe(statusStr)) + } + } + sb.Printf("epoch: %d, seqs: [%d", redact.Safe(ulh.txn.Epoch), redact.Safe(ulh.seqs[0])) + for j := 1; j < len(ulh.seqs); j++ { + sb.Printf(", %d", redact.Safe(ulh.seqs[j])) + } + sb.SafeString("]") +} + +// Information about a lock holder for replicated locks. Notably, unlike +// unreplicated locks, this does not include any sequence numbers. +type replicatedLockHolderInfo struct { + // nil if there is no holder. Else this is the TxnMeta of the latest call to + // acquire/update the lock by this transaction. For a given transaction if + // the lock is continuously held by a succession of different TxnMetas, the + // epoch must be monotonic and the ts (derived from txn.WriteTimestamp for + // some calls, and request.ts for other calls) must be monotonic. After ts + // is initialized, the timestamps inside txn are not used. + // + // TODO(arul): we don't really need to track the entire txnMeta here. We + // mostly use it as a proxy for whether the lock is held or not -- a simple + // boolean would do. + txn *enginepb.TxnMeta + + // The timestamp at which the replicated lock is held. + ts hlc.Timestamp +} + +func (rlh *replicatedLockHolderInfo) isEmpty() bool { + return rlh.txn == nil && rlh.ts.IsEmpty() +} + +func (rlh *replicatedLockHolderInfo) safeFormat( + sb *redact.StringBuilder, txnStatusCache *txnStatusCache, +) { + if rlh.txn == nil { + return + } + sb.SafeString("repl ") + if txnStatusCache != nil { + finalizedTxn, ok := txnStatusCache.finalizedTxns.get(rlh.txn.ID) + if ok { + var statusStr string + switch finalizedTxn.Status { + case roachpb.COMMITTED: + statusStr = "committed" + case roachpb.ABORTED: + statusStr = "aborted" + } + sb.Printf("[holder finalized: %s] ", redact.Safe(statusStr)) + } + } + sb.Printf("epoch: %d", redact.Safe(rlh.txn.Epoch)) } // Per lock state in lockTableImpl. @@ -975,7 +1048,18 @@ type lockState struct { holder struct { locked bool // Lock strength is always lock.Intent. - holder [lock.MaxDurability + 1]lockHolderInfo + + // INVARIANT: If the lock is held (i.e. the locked boolean is set to true), + // then atleast one of (and possibly both of) unreplicatedInfo and + // replicatedInfo must track lock holder information. + + // unreplicatedInfo tracks lock holder information if the lock is held with + // durability unreplicated. + unreplicatedInfo unreplicatedLockHolderInfo + + // replicatedInfo tracks lock holder information if the lock is held with + // durability replicated. + replicatedInfo replicatedLockHolderInfo // The start time of the lockholder being marked as held in the lock table. // NB: In the case of a replicated lock that is held by a transaction, if @@ -1157,48 +1241,20 @@ func (l *lockState) safeFormat(sb *redact.StringBuilder, txnStatusCache *txnStat sb.SafeString(" empty\n") return } - writeHolderInfo := func(sb *redact.StringBuilder, txn *enginepb.TxnMeta, ts hlc.Timestamp) { + txn, ts := l.getLockHolder() + if txn != nil { // lock is held sb.Printf(" holder: txn: %v, ts: %v, info: ", redact.Safe(txn.ID), redact.Safe(ts)) - first := true - for i := range l.holder.holder { - h := &l.holder.holder[i] - if h.txn == nil { - continue - } - if !first { - sb.SafeString(", ") - } - first = false - if lock.Durability(i) == lock.Replicated { - sb.SafeString("repl ") - } else { - sb.SafeString("unrepl ") - } - if txnStatusCache != nil { - finalizedTxn, ok := txnStatusCache.finalizedTxns.get(h.txn.ID) - if ok { - var statusStr string - switch finalizedTxn.Status { - case roachpb.COMMITTED: - statusStr = "committed" - case roachpb.ABORTED: - statusStr = "aborted" - } - sb.Printf("[holder finalized: %s] ", redact.Safe(statusStr)) - } + if !l.holder.replicatedInfo.isEmpty() { + l.holder.replicatedInfo.safeFormat(sb, txnStatusCache) + if !l.holder.unreplicatedInfo.isEmpty() { + sb.Printf(", ") } - sb.Printf("epoch: %d, seqs: [%d", redact.Safe(h.txn.Epoch), redact.Safe(h.seqs[0])) - for j := 1; j < len(h.seqs); j++ { - sb.Printf(", %d", redact.Safe(h.seqs[j])) - } - sb.SafeString("]") + } + if !l.holder.unreplicatedInfo.isEmpty() { + l.holder.unreplicatedInfo.safeFormat(sb, txnStatusCache) } sb.SafeString("\n") } - txn, ts := l.getLockHolder() - if txn != nil { - writeHolderInfo(sb, txn, ts) - } // TODO(sumeer): Add an optional `description string` field to Request and // lockTableGuardImpl that tests can set to avoid relying on the seqNum to // identify requests. @@ -1271,11 +1327,11 @@ func (l *lockState) lockStateInfo(now time.Time) roachpb.LockStateInfo { durability := lock.Unreplicated if l.holder.locked { - if l.holder.holder[lock.Replicated].txn != nil { + if l.holder.replicatedInfo.txn != nil { durability = lock.Replicated - txnHolder = l.holder.holder[lock.Replicated].txn - } else if l.holder.holder[lock.Unreplicated].txn != nil { - txnHolder = l.holder.holder[lock.Unreplicated].txn + txnHolder = l.holder.replicatedInfo.txn + } else if l.holder.unreplicatedInfo.txn != nil { + txnHolder = l.holder.unreplicatedInfo.txn } } @@ -1542,9 +1598,8 @@ func (l *lockState) isEmptyLock() bool { // The lock isn't held. Sanity check the lock state is sane: // 1. Lock holder information should be zero-ed out. // 2. There should be no waiting readers. - for i := range l.holder.holder { - assert(l.holder.holder[i].isEmpty(), "lockState with !locked but non-zero lockHolderInfo") - } + assert(l.holder.unreplicatedInfo.isEmpty(), "lockState !locked but non-zero unreplicatedInfo") + assert(l.holder.replicatedInfo.isEmpty(), "lockState !locked but non-zero replicatedInfo") assert(l.waitingReaders.Len() == 0, "lockState with waiting readers but no holder") // Determine if the lock is empty or not by checking the list of queued // writers. @@ -1616,10 +1671,10 @@ func (l *lockState) totalAndMaxWaitDuration(now time.Time) (time.Duration, time. func (l *lockState) isLockedBy(id uuid.UUID) bool { if l.holder.locked { var holderID uuid.UUID - if l.holder.holder[lock.Unreplicated].txn != nil { - holderID = l.holder.holder[lock.Unreplicated].txn.ID + if l.holder.unreplicatedInfo.txn != nil { + holderID = l.holder.unreplicatedInfo.txn.ID } else { - holderID = l.holder.holder[lock.Replicated].txn.ID + holderID = l.holder.replicatedInfo.txn.ID } return id == holderID } @@ -1634,21 +1689,22 @@ func (l *lockState) getLockHolder() (*enginepb.TxnMeta, hlc.Timestamp) { return nil, hlc.Timestamp{} } + assert( + l.holder.replicatedInfo.txn != nil || l.holder.unreplicatedInfo.txn != nil, + "lock held, but no replicated or unreplicated lock holder info", + ) + // If the lock is held as both replicated and unreplicated we want to // provide the lower of the two timestamps, since the lower timestamp // contends with more transactions. Else we provide whichever one it is held // at. - - // Start with the assumption that it is held as replicated. - index := lock.Replicated - // Condition under which we prefer the unreplicated holder. - if l.holder.holder[index].txn == nil || (l.holder.holder[lock.Unreplicated].txn != nil && + if l.holder.replicatedInfo.txn == nil || (l.holder.unreplicatedInfo.txn != nil && // If we are evaluating the following clause we are sure that it is held // as both replicated and unreplicated. - l.holder.holder[lock.Unreplicated].ts.Less(l.holder.holder[lock.Replicated].ts)) { - index = lock.Unreplicated + l.holder.unreplicatedInfo.ts.Less(l.holder.replicatedInfo.ts)) { + return l.holder.unreplicatedInfo.txn, l.holder.unreplicatedInfo.ts } - return l.holder.holder[index].txn, l.holder.holder[index].ts + return l.holder.replicatedInfo.txn, l.holder.replicatedInfo.ts } // getLockMode returns the Mode with which a lock is held. @@ -1666,9 +1722,8 @@ func (l *lockState) getLockMode() lock.Mode { func (l *lockState) clearLockHolder() { l.holder.locked = false l.holder.startTime = time.Time{} - for i := range l.holder.holder { - l.holder.holder[i] = lockHolderInfo{} - } + l.holder.replicatedInfo = replicatedLockHolderInfo{} + l.holder.unreplicatedInfo = unreplicatedLockHolderInfo{} } // scanAndMaybeEnqueue scans all locks held on the receiver's key and performs @@ -1832,7 +1887,7 @@ func (l *lockState) conflictsWithLockHolder(g *lockTableGuardImpl) bool { up := roachpb.MakeLockUpdate(finalizedTxn, roachpb.Span{Key: l.key}) // The lock belongs to a finalized transaction. There's no conflict, but the // lock must be resolved -- accumulate it on the appropriate slice. - if l.holder.holder[lock.Replicated].txn == nil { + if l.holder.replicatedInfo.txn == nil { // only held unreplicated g.toResolveUnreplicated = append(g.toResolveUnreplicated, up) } else { g.toResolve = append(g.toResolve, up) @@ -1860,7 +1915,7 @@ func (l *lockState) conflictsWithLockHolder(g *lockTableGuardImpl) bool { pushedTxn, ok := g.lt.txnStatusCache.pendingTxns.get(lockHolderTxn.ID) if ok && g.ts.Less(pushedTxn.WriteTimestamp) { up := roachpb.MakeLockUpdate(pushedTxn, roachpb.Span{Key: l.key}) - if l.holder.holder[lock.Replicated].txn == nil { + if l.holder.replicatedInfo.txn == nil { // Only held unreplicated. Accumulate it as an unreplicated lock to // resolve, in case any other waiting readers can benefit from the // pushed timestamp. @@ -2207,75 +2262,54 @@ func (l *lockState) acquireLock(acq *roachpb.LockAcquisition, clock *hlc.Clock) if acq.Txn.ID != beforeTxn.ID { return errors.AssertionFailedf("existing lock cannot be acquired by different transaction") } - if acq.Durability == lock.Unreplicated && - l.holder.holder[lock.Unreplicated].txn != nil && - l.holder.holder[lock.Unreplicated].txn.Epoch > acq.Txn.Epoch { - // If the lock is being re-acquired as an unreplicated lock, and the - // request trying to do so belongs to a prior epoch, we reject the - // request. This parallels the logic mvccPutInternal has for intents. - return errors.Errorf( - "locking request with epoch %d came after lock(unreplicated) had already been acquired at epoch %d in txn %s", - acq.Txn.Epoch, l.holder.holder[acq.Durability].txn.Epoch, acq.Txn.ID, - ) - } // TODO(arul): Once we stop storing sequence numbers/transaction protos // associated with replicated locks, the following logic can be deleted. if acq.Durability == lock.Replicated && - l.holder.holder[lock.Replicated].txn != nil && - l.holder.holder[lock.Replicated].txn.Epoch > acq.Txn.Epoch { + l.holder.replicatedInfo.txn != nil && + l.holder.replicatedInfo.txn.Epoch > acq.Txn.Epoch { // If we're dealing with a replicated lock (intent), and the transaction // acquiring this lock belongs to a prior epoch, we expect mvccPutInternal // to return an error. As such, the request should never call into // AcquireLock and reach this point. return errors.AssertionFailedf( "locking request with epoch %d came after lock(replicated) had already been acquired at epoch %d in txn %s", - acq.Txn.Epoch, l.holder.holder[acq.Durability].txn.Epoch, acq.Txn.ID, + acq.Txn.Epoch, l.holder.replicatedInfo.txn.Epoch, acq.Txn.ID, ) } - seqs := l.holder.holder[acq.Durability].seqs - // Lock is being re-acquired... - if l.holder.holder[acq.Durability].txn != nil && - // ...at a higher epoch. - l.holder.holder[acq.Durability].txn.Epoch < acq.Txn.Epoch { - // Clear the sequences for the older epoch. - seqs = seqs[:0] - } - // Lock is being re-acquired with durability Unreplicated... - if acq.Durability == lock.Unreplicated && l.holder.holder[lock.Unreplicated].txn != nil && - // ... at the same epoch. - l.holder.holder[lock.Unreplicated].txn.Epoch == acq.Txn.Epoch { - // Prune the list of sequence numbers tracked for this lock by removing - // any sequence numbers that are considered ignored by virtue of a - // savepoint rollback. - // - // Note that the in-memory lock table is the source of truth for just - // unreplicated locks, so we only do this pruning for unreplicated lock - // acquisition. On the other hand, for replicated locks, the source of - // truth is what's written in MVCC. We could try and mimic that logic - // here, but we choose not to, as doing so is error-prone/difficult to - // maintain. - seqs = removeIgnored(seqs, acq.IgnoredSeqNums) - } - - if len(seqs) > 0 && seqs[len(seqs)-1] >= acq.Txn.Sequence { - // Idempotent lock acquisition. In this case, we simply ignore the lock - // acquisition as long as it corresponds to an existing sequence number. - // If the sequence number is not being tracked yet, insert it into the - // sequence history. The validity of such a lock re-acquisition should - // have already been determined at the MVCC level. - if i := sort.Search(len(seqs), func(i int) bool { - return seqs[i] >= acq.Txn.Sequence - }); i == len(seqs) { - panic("lockTable bug - search value <= last element") - } else if seqs[i] != acq.Txn.Sequence { - seqs = append(seqs, 0) - copy(seqs[i+1:], seqs[i:]) - seqs[i] = acq.Txn.Sequence - l.holder.holder[acq.Durability].seqs = seqs + // An unreplicated lock is being re-acquired... + if acq.Durability == lock.Unreplicated && l.holder.unreplicatedInfo.txn != nil { + switch { + case l.holder.unreplicatedInfo.txn.Epoch < acq.Txn.Epoch: // at a higher epoch + // Clear sequence numbers from the older epoch. + l.holder.unreplicatedInfo.seqs = l.holder.unreplicatedInfo.seqs[:0] + case l.holder.unreplicatedInfo.txn.Epoch == acq.Txn.Epoch: // at the same epoch + // Prune the list of sequence numbers tracked for this lock by removing + // any sequence numbers that are considered ignored by virtue of a + // savepoint rollback. + // + // Note that the in-memory lock table is the source of truth for just + // unreplicated locks, so we only do this pruning for unreplicated lock + // acquisition. On the other hand, for replicated locks, the source of + // truth is what's written in MVCC. We could try and mimic that logic + // here, but we choose not to, as doing so is error-prone/difficult to + // maintain. + l.holder.unreplicatedInfo.seqs = removeIgnored( + l.holder.unreplicatedInfo.seqs, acq.IgnoredSeqNums, + ) + case l.holder.unreplicatedInfo.txn.Epoch > acq.Txn.Epoch: // at a prior epoch + // Reject the request; the logic here parallels how mvccPutInternal + // handles this case for intents. + return errors.Errorf( + "locking request with epoch %d came after lock(unreplicated) had already been acquired at epoch %d in txn %s", + acq.Txn.Epoch, l.holder.unreplicatedInfo.txn.Epoch, acq.Txn.ID, + ) + default: + panic("unreachable") } - return nil } - l.holder.holder[acq.Durability].txn = &acq.Txn + if l.isIdempotentLockAcquisition(acq) { + return nil // nothing more to do here. + } // Forward the lock's timestamp instead of assigning to it blindly. // While lock acquisition uses monotonically increasing timestamps // from the perspective of the transaction's coordinator, this does @@ -2314,8 +2348,17 @@ func (l *lockState) acquireLock(acq *roachpb.LockAcquisition, clock *hlc.Clock) // timestamp at that point, which may cause them to conflict with the // lock even if they had not conflicted before. In a sense, it is no // different than the first time a lock is added to the lockTable. - l.holder.holder[acq.Durability].ts.Forward(acq.Txn.WriteTimestamp) - l.holder.holder[acq.Durability].seqs = append(seqs, acq.Txn.Sequence) + switch acq.Durability { + case lock.Unreplicated: + l.holder.unreplicatedInfo.txn = &acq.Txn + l.holder.unreplicatedInfo.ts.Forward(acq.Txn.WriteTimestamp) + l.holder.unreplicatedInfo.seqs = append(l.holder.unreplicatedInfo.seqs, acq.Txn.Sequence) + case lock.Replicated: + l.holder.replicatedInfo.txn = &acq.Txn + l.holder.replicatedInfo.ts.Forward(acq.Txn.WriteTimestamp) + default: + panic(fmt.Sprintf("unknown lock durability: %s", acq.Durability)) + } _, afterTs := l.getLockHolder() if beforeTs.Less(afterTs) { @@ -2356,16 +2399,74 @@ func (l *lockState) acquireLock(acq *roachpb.LockAcquisition, clock *hlc.Clock) } l.holder.locked = true - l.holder.holder[acq.Durability].txn = &acq.Txn - l.holder.holder[acq.Durability].ts = acq.Txn.WriteTimestamp - l.holder.holder[acq.Durability].seqs = append([]enginepb.TxnSeq(nil), acq.Txn.Sequence) l.holder.startTime = clock.PhysicalTime() + switch acq.Durability { + case lock.Unreplicated: + l.holder.unreplicatedInfo.txn = &acq.Txn + l.holder.unreplicatedInfo.ts = acq.Txn.WriteTimestamp + l.holder.unreplicatedInfo.seqs = append([]enginepb.TxnSeq(nil), acq.Txn.Sequence) + case lock.Replicated: + l.holder.replicatedInfo.txn = &acq.Txn + l.holder.replicatedInfo.ts = acq.Txn.WriteTimestamp + default: + panic(fmt.Sprintf("unknown lock durability: %s", acq.Durability)) + } // Inform active waiters since lock has transitioned to held. l.informActiveWaiters() return nil } +// isIdempotentLockAcquisition returns true if the lock acquisition is +// idempotent. Idempotent lock acquisitions do not require any changes to what +// is being tracked in the lock's state. +// +// REQUIRES: l.mu to be locked. +func (l *lockState) isIdempotentLockAcquisition(acq *roachpb.LockAcquisition) bool { + txn, _ := l.getLockHolder() + assert(txn.ID == acq.Txn.ID, "existing lock transaction is different from the acquisition") + switch acq.Durability { + case lock.Unreplicated: + seqs := l.holder.unreplicatedInfo.seqs + // Cheaply check if this could be an idempotent lock acquisition. + if len(seqs) > 0 && seqs[len(seqs)-1] >= acq.Txn.Sequence { + // Idempotent lock acquisition. In this case, we simply ignore the lock + // acquisition as long as it corresponds to an existing sequence number. + // If the sequence number is not being tracked yet, insert it into the + // sequence history. The validity of such a lock re-acquisition should + // have already been determined at the MVCC level. + + if i := sort.Search(len(seqs), func(i int) bool { + return seqs[i] >= acq.Txn.Sequence + }); i == len(seqs) { + panic("lockTable bug - search value <= last element") + } else if seqs[i] != acq.Txn.Sequence { + // TODO(arul): Once we change the lockState datastructure to only track + // the highest sequence number, we should remove all mutations happening + // inside this function. + seqs = append(seqs, 0) + copy(seqs[i+1:], seqs[i:]) + seqs[i] = acq.Txn.Sequence + l.holder.unreplicatedInfo.seqs = seqs + } + // NB: Lock re-acquisitions at different timestamps are not considered + // idempotent. Strictly speaking, we could tighten this condition to + // consider lock re-acquisition at lower timestamps idempotent, as a + // lock's timestamp at a given durability never regresses. + return l.holder.unreplicatedInfo.ts.Equal(acq.Txn.WriteTimestamp) + } + return false + case lock.Replicated: + // NB: Lock re-acquisitions at different timestamps are not considered + // idempotent. Strictly speaking, we could tighten this condition to + // consider lock re-acquisition at lower timestamps idempotent, as a + // lock's timestamp at a given durability never regresses. + return l.holder.replicatedInfo.txn != nil && l.holder.replicatedInfo.ts.Equal(acq.Txn.WriteTimestamp) + default: + panic(fmt.Sprintf("unknown lock durability: %s", acq.Durability)) + } +} + // A replicated lock held by txn with timestamp ts was discovered by guard g // where g is trying to access this key with strength accessStrength. // Acquires l.mu. @@ -2393,11 +2494,10 @@ func (l *lockState) discoveredLock( l.holder.locked = true l.holder.startTime = clock.PhysicalTime() } - holder := &l.holder.holder[lock.Replicated] + holder := &l.holder.replicatedInfo if holder.txn == nil { holder.txn = txn holder.ts = ts - holder.seqs = append(holder.seqs, txn.Sequence) } switch accessStrength { @@ -2476,7 +2576,7 @@ func (l *lockState) tryClearLock(force bool) bool { // Clear lock holder. While doing so, construct the closure used to transition // waiters. lockHolderTxn, _ := l.getLockHolder() // only needed if this is a replicated lock - replicatedHeld := l.holder.locked && l.holder.holder[lock.Replicated].txn != nil + replicatedHeld := l.holder.locked && l.holder.replicatedInfo.txn != nil transitionWaiter := func(g *lockTableGuardImpl) { if replicatedHeld && !force { // Note that none of the current waiters can be requests from @@ -2588,51 +2688,59 @@ func (l *lockState) tryUpdateLockLocked(up roachpb.LockUpdate) (heldByTxn, gc bo _, beforeTs := l.getLockHolder() advancedTs := beforeTs.Less(ts) isLocked := false - for i := range l.holder.holder { - holder := &l.holder.holder[i] - if holder.txn == nil { - continue - } - // Note that mvccResolveWriteIntent() has special handling of the case - // where the pusher is using an epoch lower than the epoch of the intent - // (replicated lock), but is trying to push to a higher timestamp. The - // replicated lock gets written with the newer epoch (not the epoch known - // to the pusher) but a higher timestamp. Then the pusher will call into - // this function with that lower epoch. Instead of trying to be consistent - // with mvccResolveWriteIntent() in the current state of the replicated - // lock we simply forget the replicated lock since it is no longer in the - // way of this request. Eventually, once we have segregated locks, the - // lock table will be the source of truth for replicated locks too, and - // this forgetting behavior will go away. - // - // For unreplicated locks the lock table is the source of truth, so we - // best-effort mirror the behavior of mvccResolveWriteIntent() by updating - // the timestamp. - if lock.Durability(i) == lock.Replicated || txn.Epoch > holder.txn.Epoch { - *holder = lockHolderInfo{} - continue - } - // Unreplicated lock held in same epoch or a higher epoch. - if advancedTs { - // We may advance ts here but not update the holder.txn object below - // for the reason stated in the comment about mvccResolveWriteIntent(). - // The lockHolderInfo.ts is the source of truth regarding the timestamp - // of the lock, and not TxnMeta.WriteTimestamp. - holder.ts = ts - } - if txn.Epoch == holder.txn.Epoch { - holder.seqs = removeIgnored(holder.seqs, up.IgnoredSeqNums) - if len(holder.seqs) == 0 { - *holder = lockHolderInfo{} - continue + // The MVCC keyspace is the source of truth about the disposition of a + // replicated lock. Replicated locks are updated using + // mvccResolveWriteIntent(). Trying to be consistent[1] with the handling in + // mvccResolveWriteIntent() is error-prone, especially as things evolve. + // Instead of attempting to do so, we simply forget the replicated lock. + // + // [1] For a little flavour of the complexity here, mvccResolveWriteIntent() + // has special handling of the case where a pusher is using an epoch lower + // than the epoch of the intent. The intent is written at the newer epoch (the + // one not known to the pusher) but at a higher timestamp. The pusher will + // then call into this function with that lower epoch. + if l.holder.replicatedInfo.txn != nil { + l.holder.replicatedInfo = replicatedLockHolderInfo{} + } + // However, for unreplicated locks, the lock table is the source of truth. + // As such, we best-effort mirror the behavior of mvccResolveWriteIntent(). + if l.holder.unreplicatedInfo.txn != nil { + switch { + //...update corresponds to a higher epoch. + case txn.Epoch > l.holder.unreplicatedInfo.txn.Epoch: + // Forget what was tracked previously. + l.holder.unreplicatedInfo = unreplicatedLockHolderInfo{} + + // ...update corresponds to the current epoch. + case txn.Epoch == l.holder.unreplicatedInfo.txn.Epoch: + l.holder.unreplicatedInfo.seqs = removeIgnored(l.holder.unreplicatedInfo.seqs, up.IgnoredSeqNums) + if len(l.holder.unreplicatedInfo.seqs) == 0 { + l.holder.unreplicatedInfo = unreplicatedLockHolderInfo{} + isLocked = false + break } if advancedTs { - holder.txn = txn + // NB: Unlike the case below, where we can't update the txn object + // because of the desire to best-effort mirror mvccResolveWriteIntent + // internal, we can do so here because the epochs are the same. + l.holder.unreplicatedInfo.txn = txn + l.holder.unreplicatedInfo.ts = ts + } + isLocked = true + + // ...update corresponds to an older epoch of the transaction. + case txn.Epoch < l.holder.unreplicatedInfo.txn.Epoch: + if advancedTs { + // We may advance ts here but not update the holder.txn object below for + // the reason stated in the comment about mvccResolveWriteIntent(). The + // {unreplicated,replicated}LockHolderInfo.ts is the source of truth + // regarding the timestamp of the lock, and not TxnMeta.WriteTimestamp. + l.holder.unreplicatedInfo.ts = ts } + isLocked = true + default: + panic("unreachable") } - // Else txn.Epoch < lockHolderTxn.Epoch, so only the timestamp has been - // potentially updated. - isLocked = true } if !isLocked { @@ -2796,7 +2904,7 @@ func (l *lockState) tryFreeLockOnReplicatedAcquire() bool { defer l.mu.Unlock() // Bail if not locked with only the Unreplicated durability. - if !l.holder.locked || l.holder.holder[lock.Replicated].txn != nil { + if !l.holder.locked || l.holder.replicatedInfo.txn != nil { return false } diff --git a/pkg/kv/kvserver/concurrency/lock_table_test.go b/pkg/kv/kvserver/concurrency/lock_table_test.go index 25b0a667f4f2..25e01e7ca53e 100644 --- a/pkg/kv/kvserver/concurrency/lock_table_test.go +++ b/pkg/kv/kvserver/concurrency/lock_table_test.go @@ -1829,16 +1829,17 @@ func TestLockStateSafeFormat(t *testing.T) { endKey: []byte("END"), } l.holder.locked = true - l.holder.holder[lock.Replicated] = lockHolderInfo{ + // TODO(arul): add something about replicated locks here too. + l.holder.unreplicatedInfo = unreplicatedLockHolderInfo{ txn: &enginepb.TxnMeta{ID: uuid.NamespaceDNS}, ts: hlc.Timestamp{WallTime: 123, Logical: 7}, seqs: []enginepb.TxnSeq{1}, } require.EqualValues(t, - " lock: ‹\"KEY\"›\n holder: txn: 6ba7b810-9dad-11d1-80b4-00c04fd430c8, ts: 0.000000123,7, info: repl epoch: 0, seqs: [1]\n", + " lock: ‹\"KEY\"›\n holder: txn: 6ba7b810-9dad-11d1-80b4-00c04fd430c8, ts: 0.000000123,7, info: unrepl epoch: 0, seqs: [1]\n", redact.Sprint(l)) require.EqualValues(t, - " lock: ‹×›\n holder: txn: 6ba7b810-9dad-11d1-80b4-00c04fd430c8, ts: 0.000000123,7, info: repl epoch: 0, seqs: [1]\n", + " lock: ‹×›\n holder: txn: 6ba7b810-9dad-11d1-80b4-00c04fd430c8, ts: 0.000000123,7, info: unrepl epoch: 0, seqs: [1]\n", redact.Sprint(l).Redact()) } 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 9ecea737ddb1..ad6453081c44 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents @@ -37,25 +37,25 @@ debug-lock-table ---- num=10 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -72,28 +72,28 @@ debug-lock-table ---- num=10 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 1, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 1 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 debug-advance-clock ts=123 ---- @@ -164,7 +164,7 @@ debug-lock-table ---- num=1 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 @@ -204,7 +204,7 @@ num=2 queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0 queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 @@ -236,7 +236,7 @@ num=3 queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: committed] epoch: 0 queued writers: active: false req: 2, txn: 00000001-0000-0000-0000-000000000000 @@ -306,9 +306,9 @@ debug-lock-table ---- num=2 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 new-request name=req2 txn=txn2 ts=10,1 put key=g value=v1 @@ -338,9 +338,9 @@ debug-lock-table ---- num=4 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "h" @@ -361,12 +361,12 @@ debug-lock-table ---- num=4 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 3, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 3 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "h" @@ -449,15 +449,15 @@ debug-lock-table ---- num=3 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "d" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "e" - holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 @@ -476,16 +476,16 @@ debug-lock-table ---- num=3 lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 lock: "d" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "e" - holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 @@ -535,16 +535,16 @@ num=5 lock: "b" holder: txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 lock: "d" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "e" - holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 @@ -575,16 +575,16 @@ num=5 lock: "b" holder: txn: 00000004-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 lock: "d" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "e" - holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 @@ -621,11 +621,11 @@ num=4 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "d" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "e" - holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 @@ -657,7 +657,7 @@ num=3 queued writers: active: false req: 5, txn: 00000001-0000-0000-0000-000000000000 lock: "e" - holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000005-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 5, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 5 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/clear_abandoned_intents_without_adding_to_lock_table index 7f32378032ca..c850b40fd252 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 @@ -34,7 +34,7 @@ debug-lock-table ---- num=1 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -51,7 +51,7 @@ debug-lock-table ---- num=1 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 1, txn: 00000001-0000-0000-0000-000000000000 distinguished req: 1 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 f7763af0ce33..8ce068bd7185 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 @@ -138,7 +138,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0 sequence req=req4 ---- @@ -161,7 +161,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,0, info: repl epoch: 0 waiting readers: req: 3, txn: 00000004-0000-0000-0000-000000000000 distinguished req: 3 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock index c921548aee2b..5288418d743a 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/discovered_lock @@ -29,7 +29,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 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 1812eac9491c..aa2de901a390 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/lock_timeout @@ -215,7 +215,7 @@ num=3 active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k4" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0 # ------------------------------------------------------------- # Read-only request with lock timeout discovers abandoned @@ -273,7 +273,7 @@ debug-lock-table ---- num=1 lock: "k4" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 reset ---- 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 c35c14595e00..30052d371b2d 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/range_state_listener @@ -164,7 +164,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000002-0000-0000-0000-000000000000 @@ -279,7 +279,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 4, txn: 00000003-0000-0000-0000-000000000000 @@ -437,7 +437,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 6, txn: 00000002-0000-0000-0000-000000000000 @@ -657,7 +657,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 10, txn: 00000002-0000-0000-0000-000000000000 @@ -815,7 +815,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 12, txn: 00000002-0000-0000-0000-000000000000 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents index 5f5873a17b31..89017dea6153 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents @@ -40,25 +40,25 @@ debug-lock-table ---- num=10 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 # Before re-scanning and pushing, add a waiter on a single key to demonstrate # that uncontended, replicated keys are released when pushed, while contended, @@ -168,9 +168,9 @@ debug-lock-table ---- num=2 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 new-request name=req2 txn=txn2 ts=10,1 put key=g value=v1 @@ -200,9 +200,9 @@ debug-lock-table ---- num=4 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "h" @@ -276,11 +276,11 @@ debug-lock-table ---- num=3 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -366,25 +366,25 @@ debug-lock-table ---- num=10 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 # Before re-scanning and pushing, add a waiter on a single key to demonstrate # that uncontended, replicated keys are released when pushed, while contended, diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table index bb49bde8d966..4419814e6bb7 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table @@ -34,7 +34,7 @@ debug-lock-table ---- num=1 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -146,7 +146,7 @@ debug-lock-table ---- num=1 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -202,23 +202,23 @@ debug-lock-table ---- num=9 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req2 ---- @@ -317,7 +317,7 @@ debug-lock-table ---- num=1 lock: "a" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -373,23 +373,23 @@ debug-lock-table ---- num=9 lock: "b" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "d" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "e" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "f" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "g" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "h" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "i" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 lock: "j" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req2 ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty index c60eb0762fd0..2d3c9c330d1c 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/uncertainty @@ -31,7 +31,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -97,7 +97,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 100.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 100.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -166,7 +166,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 100.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 100.000000000,1, info: repl epoch: 0 sequence req=req1 ---- @@ -241,7 +241,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 14.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 14.000000000,1, info: repl epoch: 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 6b7d64192911..ce4aded2434f 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/update @@ -381,7 +381,7 @@ debug-lock-table ---- 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] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] queued writers: active: true req: 9, txn: none distinguished req: 9 diff --git a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere index 607027c81620..8a8fdce16611 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_elsewhere @@ -69,7 +69,7 @@ debug-lock-table ---- num=1 lock: "k" - holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000001-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 2, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 2 @@ -211,12 +211,12 @@ debug-lock-table ---- num=2 lock: "k1" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 5, txn: 00000002-0000-0000-0000-000000000000 distinguished req: 5 lock: "k2" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 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. @@ -242,7 +242,7 @@ debug-lock-table ---- num=1 lock: "k1" - holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000003-0000-0000-0000-000000000000, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 6, txn: 00000002-0000-0000-0000-000000000000 req: 5, txn: 00000002-0000-0000-0000-000000000000 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 b3475e946488..d0c119c8cfb3 100644 --- a/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error +++ b/pkg/kv/kvserver/concurrency/testdata/concurrency_manager/wait_policy_error @@ -215,7 +215,7 @@ num=3 active: true req: 3, txn: 00000003-0000-0000-0000-000000000000 distinguished req: 3 lock: "k4" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl epoch: 0 # ------------------------------------------------------------- # Read-only request with WaitPolicy_Error discovers abandoned @@ -276,7 +276,7 @@ debug-lock-table ---- num=1 lock: "k4" - holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000002-0000-0000-0000-000000000000, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 reset ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs index f0ae1f407bcd..ad7293192c9e 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/acquire_ignored_seqs @@ -85,9 +85,11 @@ num=1 holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [2, 5, 8] # ------------------------------------------------------------------------------ -# Ensure only sequence numbers of unreplicated locks are pruned. That is, -# replicated locks acquired at a sequence number that is considered ignored -# should not be pruned. +# Ensure sequence numbers are only pruned when acquiring unreplicated locks. +# The interesting case here is when a replicated lock is being acquired and the +# acquisition struct says a sequence number at which an unreplicated lock is +# held is considered ignored -- nothing gets pruned in such cases (even though +# it technically could). # ------------------------------------------------------------------------------ # First, add a waiting writer on this lock so that it counts as contended. @@ -102,42 +104,31 @@ scan r=req6 ---- start-waiting: true -acquire r=req5 k=a durability=r ----- -num=1 - lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: repl epoch: 0, seqs: [8], unrepl epoch: 0, seqs: [2, 5, 8] - queued writers: - active: true req: 3, txn: 00000000-0000-0000-0000-000000000002 - distinguished req: 3 - new-txn txn=txn1 ts=10,1 epoch=0 seq=9 ---- new-request r=req7 txn=txn1 ts=10,1 spans=intent@a ---- -# Note that the lock is acquired as both replicated and un-replicated at -# sequence number 8. But because the lock is being acquired as replicated, we -# don't prune the list of unreplicated locks either. +# Because the lock is being acquired as a replicated lock nothing in the +# unreplicated sequence number tracking gets pruned. acquire r=req7 k=a durability=r ignored-seqs=8 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: repl epoch: 0, seqs: [8, 9], unrepl epoch: 0, seqs: [2, 5, 8] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [2, 5, 8] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 -# Similarly, acquire the lock as unreplicated and try to ignore a sequence -# number (8) at which the lock was acquired as a replicated lock; it shouldn't -# be pruned. +# However, unreplicated lock acquisition with the same ignored sequence number +# (8) will result in the list of sequence numbers getting pruned. acquire r=req7 k=a durability=u ignored-seqs=8 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: repl epoch: 0, seqs: [8, 9], unrepl epoch: 0, seqs: [2, 5, 9] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [2, 5, 9] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 @@ -160,7 +151,7 @@ acquire r=req8 k=a durability=u ignored-seqs=2-5,9 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: repl epoch: 0, seqs: [8, 9], unrepl epoch: 0, seqs: [11] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [11] queued writers: active: true req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered index cd77f1216895..73920c306e3d 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered @@ -89,7 +89,7 @@ add-discovered r=req2 k=a txn=txn3 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 2, txn: 00000000-0000-0000-0000-000000000002 @@ -109,7 +109,7 @@ dequeue r=req3 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 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 b4ac091b49f0..5d8f4683adde 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 @@ -29,7 +29,7 @@ add-discovered r=req1 k=b txn=txn2 lease-seq=5 ---- num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 add-discovered r=req1 k=c txn=txn2 lease-seq=6 ---- @@ -39,4 +39,4 @@ print ---- num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic index 1ce7eff46680..7449354cd486 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/basic +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/basic @@ -187,7 +187,7 @@ add-discovered r=req4 k=a txn=txn3 ---- num=4 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" @@ -203,7 +203,7 @@ add-discovered r=req4 k=f txn=txn3 ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" @@ -215,7 +215,7 @@ num=5 lock: "e" 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] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 # Note that guard state has not changed yet. Discovering these locks means the caller has to # scan again. @@ -254,7 +254,7 @@ print ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 @@ -267,7 +267,7 @@ num=5 lock: "e" 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] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 query ---- @@ -392,7 +392,7 @@ dequeue r=req5 ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 @@ -405,7 +405,7 @@ num=5 lock: "e" 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] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 # 100ms passes between req5 and req6 time-tick ms=100 @@ -440,7 +440,7 @@ print ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 @@ -455,7 +455,7 @@ num=5 lock: "e" 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] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 metrics ---- @@ -595,7 +595,7 @@ print ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 queued writers: active: true req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 @@ -612,7 +612,7 @@ num=5 lock: "e" 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] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 metrics ---- @@ -752,7 +752,7 @@ num=5 lock: "e" 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] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 guard-state r=req4 ---- @@ -781,7 +781,7 @@ num=5 lock: "e" 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] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 6.000000000,0, info: repl epoch: 0 waiting readers: req: 4, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 4 @@ -955,7 +955,7 @@ num=4 queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 @@ -974,12 +974,12 @@ num=4 queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -1001,12 +1001,12 @@ num=4 queued writers: active: false req: 4, txn: 00000000-0000-0000-0000-000000000002 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -1113,12 +1113,12 @@ dequeue r=req4 ---- num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 7 @@ -1149,7 +1149,7 @@ release txn=txn2 span=c,f ---- num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 @@ -1171,7 +1171,7 @@ print ---- num=3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0 queued writers: active: true req: 6, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 6 @@ -2088,7 +2088,7 @@ acquire r=req12 k=c durability=r ---- 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 @@ -2097,7 +2097,7 @@ dequeue r=req12 ---- num=1 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,12, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] queued writers: active: true req: 11, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 11 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear index 629001d68dc8..c427a7a306d1 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/clear +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/clear @@ -68,7 +68,7 @@ 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: 8.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,1, info: repl epoch: 0 # A non-transactional read comes in at a and blocks on the lock. @@ -125,7 +125,7 @@ num=3 active: true req: 5, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 8.000000000,1, info: repl epoch: 0 # Clearing removes all locks and allows all waiting requests to proceed. 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 ec0b77e07210..fecf7c7ed669 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 @@ -47,7 +47,7 @@ add-discovered r=req1 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -55,11 +55,11 @@ add-discovered r=req1 k=b txn=txn2 ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -67,15 +67,15 @@ add-discovered r=req1 k=d txn=txn3 ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -83,19 +83,19 @@ add-discovered r=req1 k=e txn=txn3 ---- num=4 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -107,21 +107,21 @@ acquire r=req2 k=c durability=u ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -129,21 +129,21 @@ dequeue r=req2 ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -155,22 +155,22 @@ print ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -189,11 +189,11 @@ print ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" @@ -202,11 +202,11 @@ num=5 active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -217,22 +217,22 @@ release txn=txn4 span=c ---- num=5 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "c" queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl [holder finalized: committed] epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl [holder finalized: committed] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 11.000000000,1, info: repl [holder finalized: committed] epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -295,7 +295,7 @@ add-discovered r=req3 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 @@ -303,11 +303,11 @@ add-discovered r=req3 k=c txn=txn2 ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 @@ -319,13 +319,13 @@ acquire r=req4 k=b durability=u ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 @@ -333,13 +333,13 @@ dequeue r=req4 ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 3, txn: 00000000-0000-0000-0000-000000000001 @@ -398,7 +398,7 @@ add-discovered r=req5 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 @@ -406,11 +406,11 @@ add-discovered r=req5 k=b txn=txn2 ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 @@ -422,12 +422,12 @@ print ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 5 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 @@ -450,13 +450,13 @@ print ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: true req: 5, txn: 00000000-0000-0000-0000-000000000001 active: true req: 6, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 5 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 @@ -469,7 +469,7 @@ num=2 active: true req: 6, txn: 00000000-0000-0000-0000-000000000003 distinguished req: 6 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 5, txn: 00000000-0000-0000-0000-000000000001 @@ -530,7 +530,7 @@ add-discovered r=req7 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 @@ -538,11 +538,11 @@ add-discovered r=req7 k=b txn=txn2 ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 @@ -558,12 +558,12 @@ print ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 7 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 7, txn: 00000000-0000-0000-0000-000000000001 @@ -627,37 +627,37 @@ add-discovered r=req9 k=a txn=txn3 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0 add-discovered r=req9 k=b txn=txn3 ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0 add-discovered r=req9 k=c txn=txn4 ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0 add-discovered r=req9 k=d txn=txn4 ---- num=4 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0 pushed-txn-updated txn=txn3 status=aborted ---- @@ -674,16 +674,16 @@ print ---- num=4 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl [holder finalized: aborted] epoch: 0 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000003, ts: 12.000000000,1, info: repl [holder finalized: aborted] epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0 waiting readers: req: 9, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 9 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl epoch: 0 scan r=req10 ---- @@ -703,7 +703,7 @@ release txn=txn4 span=c ---- num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 guard-state r=req9 ---- @@ -733,7 +733,7 @@ add-discovered r=req11 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 12.000000000,1, info: repl epoch: 0 queued writers: active: false req: 11, txn: none @@ -778,7 +778,7 @@ add-discovered r=req12 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 12.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 12.000000000,1, info: repl epoch: 0 pushed-txn-updated txn=txn2 status=aborted ---- @@ -1005,7 +1005,7 @@ add-discovered r=req17 k=a txn=txn7 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 16, txn: 00000000-0000-0000-0000-000000000005 @@ -1013,11 +1013,11 @@ add-discovered r=req17 k=b txn=txn8 ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 16, txn: 00000000-0000-0000-0000-000000000005 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 16, txn: 00000000-0000-0000-0000-000000000005 @@ -1029,12 +1029,12 @@ print ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: true req: 16, txn: 00000000-0000-0000-0000-000000000005 distinguished req: 16 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: false req: 16, txn: 00000000-0000-0000-0000-000000000005 @@ -1055,11 +1055,11 @@ print ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 16, txn: 00000000-0000-0000-0000-000000000005 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: true req: 16, txn: 00000000-0000-0000-0000-000000000005 distinguished req: 16 @@ -1076,13 +1076,13 @@ print ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000007, ts: 11.000000000,1, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 16, txn: 00000000-0000-0000-0000-000000000005 active: true req: 17, txn: 00000000-0000-0000-0000-000000000008 distinguished req: 17 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000008, ts: 11.000000000,1, info: repl epoch: 0 queued writers: active: true req: 16, txn: 00000000-0000-0000-0000-000000000005 distinguished req: 16 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable index 586c63fb5687..33a0fa7b8917 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/disable +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/disable @@ -73,7 +73,7 @@ add-discovered r=req2 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 queued writers: active: false req: 2, txn: 00000000-0000-0000-0000-000000000001 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 45243d771fdf..3b14b9c97216 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 @@ -40,7 +40,7 @@ add-discovered r=req1 k=a txn=txn2 consult-txn-status-cache=false ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl [holder finalized: aborted] epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl [holder finalized: aborted] epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -97,7 +97,7 @@ num=2 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -120,7 +120,7 @@ dequeue r=req1 ---- num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000004, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000004, ts: 10.000000000,0, info: repl epoch: 0 clear ---- @@ -144,7 +144,7 @@ add-discovered r=req2 k=e txn=txn5 consult-txn-status-cache=false ---- num=1 lock: "e" - holder: txn: 00000000-0000-0000-0000-000000000005, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000005, ts: 10.000000000,0, info: repl epoch: 0 # Nothing to resolve yet. resolve-before-scanning r=req2 @@ -187,7 +187,7 @@ add-discovered r=req2 k=g txn=txn7 consult-txn-status-cache=true ---- num=1 lock: "g" - holder: txn: 00000000-0000-0000-0000-000000000007, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000007, ts: 10.000000000,0, info: repl epoch: 0 # Locks for f and g were not added to lock table. resolve-before-scanning r=req2 @@ -208,7 +208,7 @@ dequeue r=req2 ---- num=1 lock: "g" - holder: txn: 00000000-0000-0000-0000-000000000007, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000007, ts: 10.000000000,0, info: repl epoch: 0 clear ---- diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes index 80ffc602dcb9..2e9816942d7b 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_changes @@ -66,7 +66,7 @@ acquire r=req2 k=a durability=r ---- 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [2, 3] queued writers: active: true req: 1, txn: none distinguished req: 1 @@ -75,7 +75,7 @@ dequeue r=reqContend ---- 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [2, 3] # --------------------------------------------------------------------------------- # Lock is reacquired at a different epoch. The old sequence numbers are discarded. @@ -91,7 +91,7 @@ acquire r=req3 k=a durability=u ---- 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, unrepl epoch: 1, seqs: [0] # --------------------------------------------------------------------------------- # Lock is reacquired at a different epoch with lower timestamp. This is allowed, @@ -109,7 +109,7 @@ acquire r=req4 k=a durability=u ---- 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, unrepl epoch: 2, seqs: [0] # --------------------------------------------------------------------------------- # Reader waits until the timestamp of the lock is updated. @@ -130,7 +130,7 @@ print ---- 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 8.000000000,0, info: repl epoch: 0, unrepl epoch: 2, seqs: [0] waiting readers: req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -145,7 +145,7 @@ acquire r=req6 k=a durability=r ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 2, seqs: [1], unrepl epoch: 2, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 2, unrepl epoch: 2, seqs: [0] waiting readers: req: 2, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 2 @@ -158,7 +158,7 @@ acquire r=req6 k=a durability=u ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 2, seqs: [1], unrepl epoch: 2, seqs: [0, 1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 2, unrepl epoch: 2, seqs: [0, 1] guard-state r=req5 ---- @@ -186,7 +186,7 @@ add-discovered r=req7 k=a txn=txn1 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 2, seqs: [1], unrepl epoch: 2, seqs: [0, 1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 2, unrepl epoch: 2, seqs: [0, 1] waiting readers: req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 @@ -211,7 +211,7 @@ print ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 2, seqs: [1], unrepl epoch: 2, seqs: [0, 1] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 14.000000000,0, info: repl epoch: 2, unrepl epoch: 2, seqs: [0, 1] waiting readers: req: 3, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 3 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped index 8902fa138b00..47f6faae5654 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/lock_dropped @@ -104,7 +104,7 @@ acquire r=req1 k=a durability=r ---- 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [2] waiting readers: req: 1, txn: none queued writers: diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter index 20a0b1114779..f1a6d1e98c50 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_active_waiter @@ -20,7 +20,7 @@ add-discovered r=req1 k=a txn=txn2 ---- num=1 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -28,23 +28,23 @@ add-discovered r=req1 k=b txn=txn2 ---- num=2 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 add-discovered r=req1 k=c txn=txn2 ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -54,13 +54,13 @@ print ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -77,13 +77,13 @@ print ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -101,14 +101,14 @@ print ---- num=3 lock: "a" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -122,9 +122,9 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -141,9 +141,9 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: true req: 1, txn: 00000000-0000-0000-0000-000000000001 active: true req: 2, txn: 00000000-0000-0000-0000-000000000001 @@ -158,7 +158,7 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 lock: "c" queued writers: active: false req: 1, txn: 00000000-0000-0000-0000-000000000001 @@ -179,7 +179,7 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 waiting readers: req: 1, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 1 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 4ffca5092bbc..ff7ee5c16ef9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/non_txn_write @@ -248,7 +248,7 @@ num=2 queued writers: active: false req: 6, txn: 00000000-0000-0000-0000-000000000001 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 4, txn: none active: false req: 5, txn: 00000000-0000-0000-0000-000000000003 diff --git a/pkg/kv/kvserver/concurrency/testdata/lock_table/resolve_pushed_txn_locks b/pkg/kv/kvserver/concurrency/testdata/lock_table/resolve_pushed_txn_locks index 0fd46373384c..f6f7fd9cdbbe 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/resolve_pushed_txn_locks +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/resolve_pushed_txn_locks @@ -36,15 +36,15 @@ add-discovered r=req4 k=b txn=txn2 ---- num=1 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 add-discovered r=req4 k=c txn=txn2 ---- num=2 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 new-request r=reqLock txn=txn2 ts=10,1 spans=intent@a+intent@b+intent@c+intent@d ---- @@ -59,9 +59,9 @@ num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 acquire r=reqLock k=b durability=u ---- @@ -69,9 +69,9 @@ num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 dequeue r=reqLock ---- @@ -79,9 +79,9 @@ num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 scan r=req1 ---- @@ -104,12 +104,12 @@ num=3 req: 3, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 3 lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 4 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 5, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 5 @@ -127,12 +127,12 @@ num=3 lock: "a" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "b" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0], unrepl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] waiting readers: req: 4, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 4 lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 5, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 5 @@ -164,7 +164,7 @@ num=3 lock: "b" holder: txn: 00000000-0000-0000-0000-000000000002, ts: 11.000000000,1, info: unrepl epoch: 0, seqs: [0] lock: "c" - holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000002, ts: 10.000000000,1, info: repl epoch: 0 waiting readers: req: 5, txn: 00000000-0000-0000-0000-000000000001 distinguished req: 5 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 3703767d349a..d88ba32bb6d9 100644 --- a/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded +++ b/pkg/kv/kvserver/concurrency/testdata/lock_table/size_limit_exceeded @@ -68,7 +68,7 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] queued writers: active: true req: 2, txn: none distinguished req: 2 @@ -81,7 +81,7 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] dequeue r=req1 ---- @@ -91,7 +91,7 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] new-request r=req2 txn=txn2 ts=10 spans=intent@a,c ---- @@ -119,7 +119,7 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] release txn=txn1 span=a ---- @@ -131,7 +131,7 @@ num=3 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] guard-state r=req2 ---- @@ -154,7 +154,7 @@ num=3 active: true req: 3, txn: 00000000-0000-0000-0000-000000000002 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] new-request r=req4 txn=txn2 ts=10 spans=none@b ---- @@ -204,12 +204,12 @@ num=4 active: true req: 6, txn: 00000000-0000-0000-0000-000000000002 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] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, unrepl epoch: 0, seqs: [0] queued writers: active: true req: 7, txn: 00000000-0000-0000-0000-000000000002 distinguished req: 7 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 8, txn: 00000000-0000-0000-0000-000000000002 @@ -226,7 +226,7 @@ acquire r=req8 k=e durability=u ---- num=1 lock: "d" - holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0, seqs: [0] + holder: txn: 00000000-0000-0000-0000-000000000001, ts: 10.000000000,0, info: repl epoch: 0 queued writers: active: false req: 8, txn: 00000000-0000-0000-0000-000000000002