Skip to content

Commit 2c48935

Browse files
craig[bot]arulajmanisumeerbhola
committed
158164: concurrency: use the correct lock strength for waiting requests r=arulajmani a=arulajmani Previously, we incorrectly assumed every locking request waiting in the lock table was doing so with Exclusive lock strength when returning output for the cluster locks table. Epic: none Release note: None 158277: mma: fix todos related to is-leaseholder assertions r=wenyihu6 a=sumeerbhola The existing assertions were valid, but needed better commentary. Also added a new assertion. Epic: CRDB-55052 Release note: None Co-authored-by: Arul Ajmani <arulajmani@gmail.com> Co-authored-by: sumeerbhola <sumeer@cockroachlabs.com>
3 parents 17a8a19 + ba2d2b2 + a05d39a commit 2c48935

File tree

10 files changed

+91
-35
lines changed

10 files changed

+91
-35
lines changed

pkg/ccl/logictestccl/testdata/logic_test/cluster_locks_tenant

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -203,7 +203,7 @@ query TTTTTTTBB colnames,rowsort
203203
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks WHERE table_name='t' AND txn_id='$txn4'
204204
----
205205
database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended
206-
test public t /Table/106/1/"b"/0 Exclusive Unreplicated SERIALIZABLE false true
206+
test public t /Table/106/1/"b"/0 Intent Unreplicated SERIALIZABLE false true
207207

208208
query I
209209
SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't'

pkg/ccl/logictestccl/testdata/logic_test/cluster_locks_tenant_write_buffering

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -206,7 +206,7 @@ query TTTTTTTBB colnames,rowsort
206206
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks WHERE table_name='t' AND txn_id='$txn4'
207207
----
208208
database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended
209-
test public t /Table/106/1/"b"/0 Exclusive Unreplicated SERIALIZABLE false true
209+
test public t /Table/106/1/"b"/0 Intent Unreplicated SERIALIZABLE false true
210210

211211
query I
212212
SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't'

pkg/kv/kvserver/allocator/mmaprototype/cluster_state_rebalance_stores.go

Lines changed: 25 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -524,8 +524,12 @@ func (re *rebalanceEnv) rebalanceLeasesFromLocalStoreID(
524524
store.StoreID, store.dimSummary[CPURate], overloadSlow)
525525
// This store is local, and cpu overloaded. Shed leases first.
526526
//
527-
// NB: any ranges at this store that don't have pending changes must
528-
// have this local store as the leaseholder.
527+
// NB: due to REQUIREMENT(change-computation), the top-k ranges for ss
528+
// reflect the latest adjusted state, including pending changes. Thus,
529+
// this store must be a replica and the leaseholder, which is asserted
530+
// below. The code below additionally ignores the range if it has pending
531+
// changes, which while not necessary for the is-leaseholder assertion,
532+
// makes the case where we assert even narrower.
529533
topKRanges := ss.adjusted.topKRanges[localStoreID]
530534
var leaseTransferCount int
531535
n := topKRanges.len()
@@ -548,37 +552,38 @@ func (re *rebalanceEnv) rebalanceLeasesFromLocalStoreID(
548552
log.KvDistribution.VEventf(ctx, 2, "skipping r%d: has pending changes", rangeID)
549553
continue
550554
}
555+
foundLocalReplica := false
551556
for _, repl := range rstate.replicas {
552557
if repl.StoreID != localStoreID { // NB: localStoreID == ss.StoreID == store.StoreID
553558
continue
554559
}
555560
if !repl.IsLeaseholder {
556-
// TODO(tbg): is this true? Can't there be ranges with replicas on
557-
// multiple local stores, and wouldn't this assertion fire in that
558-
// case once rebalanceStores is invoked on whichever of the two
559-
// stores doesn't hold the lease?
560-
//
561-
// TODO(tbg): see also the other assertion below (leaseholderID !=
562-
// store.StoreID) which seems similar to this one.
563-
log.KvDistribution.Fatalf(ctx, "internal state inconsistency: replica considered for lease shedding has no pending"+
564-
" changes but is not leaseholder: %+v", rstate)
561+
// NB: due to REQUIREMENT(change-computation), the top-k
562+
// ranges for ss reflect the latest adjusted state, including
563+
// pending changes. Thus, this store must be a replica and the
564+
// leaseholder, hence this assertion, and other assertions
565+
// below. Additionally, the code above ignored the range if it
566+
// has pending changes, which while not necessary for the
567+
// is-leaseholder assertion, makes the case where we assert
568+
// even narrower.
569+
log.KvDistribution.Fatalf(ctx,
570+
"internal state inconsistency: replica considered for lease shedding has no pending"+
571+
" changes but is not leaseholder: %+v", rstate)
565572
}
573+
foundLocalReplica = true
574+
break
575+
}
576+
if !foundLocalReplica {
577+
log.KvDistribution.Fatalf(
578+
ctx, "internal state inconsistency: local store is not a replica: %+v", rstate)
566579
}
567580
if re.now.Sub(rstate.lastFailedChange) < re.lastFailedChangeDelayDuration {
568581
log.KvDistribution.VEventf(ctx, 2, "skipping r%d: too soon after failed change", rangeID)
569582
continue
570583
}
571584
re.ensureAnalyzedConstraints(rstate)
572585
if rstate.constraints.leaseholderID != store.StoreID {
573-
// We should not panic here since the leaseQueue may have shed the
574-
// lease and informed MMA, since the last time MMA computed the
575-
// top-k ranges. This is useful for debugging in the prototype, due
576-
// to the lack of unit tests.
577-
//
578-
// TODO(tbg): can the above scenario currently happen? ComputeChanges
579-
// first processes the leaseholder message and then, still under the
580-
// lock, immediately calls into rebalanceStores (i.e. this store).
581-
// Doesn't this mean that the leaseholder view is up to date?
586+
// See the earlier comment about assertions.
582587
panic(fmt.Sprintf("internal state inconsistency: "+
583588
"store=%v range_id=%v should be leaseholder but isn't",
584589
store.StoreID, rangeID))

pkg/kv/kvserver/concurrency/lock_table.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -2069,7 +2069,7 @@ func (kl *keyLocks) lockStateInfo(now time.Time, rangeID roachpb.RangeID) []roac
20692069
lockWaiters = append(lockWaiters, lock.Waiter{
20702070
WaitingTxn: g.txnMeta(),
20712071
ActiveWaiter: qg.active,
2072-
Strength: lock.Exclusive,
2072+
Strength: qg.mode.Strength,
20732073
WaitDuration: now.Sub(g.mu.curLockWaitStart),
20742074
})
20752075
g.mu.Unlock()

pkg/kv/kvserver/concurrency/testdata/lock_table/add_discovered

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -79,8 +79,8 @@ num locks: 1, bytes returned: 79, resume reason: RESUME_UNKNOWN, resume span: <n
7979
locks:
8080
range_id=3 key="a" holder=<nil> durability=Unreplicated duration=0s
8181
waiters:
82-
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:false strength:Exclusive wait_duration:0s
83-
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Exclusive wait_duration:0s
82+
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:false strength:Intent wait_duration:0s
83+
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Intent wait_duration:0s
8484

8585
guard-state r=req2
8686
----

pkg/kv/kvserver/concurrency/testdata/lock_table/basic

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -710,11 +710,11 @@ num locks: 3, bytes returned: 282, resume reason: RESUME_UNKNOWN, resume span: <
710710
range_id=3 key="b" holder=<nil> durability=Unreplicated duration=0s
711711
waiters:
712712
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:false strength:Exclusive wait_duration:2.65s
713-
waiting_txn:00000000-0000-0000-0000-000000000001 active_waiter:true strength:Exclusive wait_duration:250ms
713+
waiting_txn:00000000-0000-0000-0000-000000000001 active_waiter:true strength:Intent wait_duration:250ms
714714
range_id=3 key="c" holder=<nil> durability=Unreplicated duration=0s
715715
waiters:
716716
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:false strength:Exclusive wait_duration:2.65s
717-
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Exclusive wait_duration:0s
717+
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Intent wait_duration:0s
718718

719719
# 100ms passes between before releasing a
720720
time-tick ms=100
@@ -872,11 +872,11 @@ num locks: 3, bytes returned: 258, resume reason: RESUME_UNKNOWN, resume span: <
872872
range_id=3 key="b" holder=<nil> durability=Unreplicated duration=0s
873873
waiters:
874874
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:false strength:Exclusive wait_duration:0s
875-
waiting_txn:00000000-0000-0000-0000-000000000001 active_waiter:true strength:Exclusive wait_duration:350ms
875+
waiting_txn:00000000-0000-0000-0000-000000000001 active_waiter:true strength:Intent wait_duration:350ms
876876
range_id=3 key="c" holder=<nil> durability=Unreplicated duration=0s
877877
waiters:
878878
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:false strength:Exclusive wait_duration:0s
879-
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Exclusive wait_duration:100ms
879+
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Intent wait_duration:100ms
880880
range_id=3 key="f" holder=00000000-0000-0000-0000-000000000003 durability=Replicated duration=2.75s
881881
waiters:
882882
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:true strength:None wait_duration:0s

pkg/kv/kvserver/concurrency/testdata/lock_table/dup_access

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -227,7 +227,7 @@ num locks: 1, bytes returned: 73, resume reason: RESUME_UNKNOWN, resume span: <n
227227
locks:
228228
range_id=3 key="c" holder=00000000-0000-0000-0000-000000000001 durability=Unreplicated duration=0s
229229
waiters:
230-
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Exclusive wait_duration:0s
230+
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Intent wait_duration:0s
231231

232232
# req5 encounters the reservation by req4 at "b" when looking at it for its read access, but ignores
233233
# it.

pkg/kv/kvserver/concurrency/testdata/lock_table/query

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -151,23 +151,23 @@ num locks: 1, bytes returned: 91, resume reason: RESUME_BYTE_LIMIT, resume span:
151151
locks:
152152
range_id=3 key="b" holder=00000000-0000-0000-0000-000000000001 durability=Unreplicated duration=200ms
153153
waiters:
154-
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:true strength:Exclusive wait_duration:200ms
154+
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:true strength:Intent wait_duration:200ms
155155

156156
query span=b max-bytes=100
157157
----
158158
num locks: 1, bytes returned: 91, resume reason: RESUME_UNKNOWN, resume span: <nil>
159159
locks:
160160
range_id=3 key="b" holder=00000000-0000-0000-0000-000000000001 durability=Unreplicated duration=200ms
161161
waiters:
162-
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:true strength:Exclusive wait_duration:200ms
162+
waiting_txn:00000000-0000-0000-0000-000000000002 active_waiter:true strength:Intent wait_duration:200ms
163163

164164
query span=e,/Max max-bytes=100
165165
----
166166
num locks: 1, bytes returned: 91, resume reason: RESUME_UNKNOWN, resume span: <nil>
167167
locks:
168168
range_id=3 key="e" holder=00000000-0000-0000-0000-000000000001 durability=Unreplicated duration=200ms
169169
waiters:
170-
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Exclusive wait_duration:200ms
170+
waiting_txn:00000000-0000-0000-0000-000000000003 active_waiter:true strength:Intent wait_duration:200ms
171171

172172
clear
173173
----

pkg/sql/logictest/testdata/logic_test/cluster_locks

Lines changed: 52 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -268,7 +268,7 @@ query TTTTTTTBB colnames
268268
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r1 AND txn_id='$txn4'
269269
----
270270
database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended
271-
test public t /Table/106/1/"b"/0 Exclusive Unreplicated SERIALIZABLE false true
271+
test public t /Table/106/1/"b"/0 Intent Unreplicated SERIALIZABLE false true
272272

273273
query TTTTTTTBB colnames,rowsort
274274
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r2 AND txn_id='$txn3'
@@ -431,3 +431,54 @@ awaitstatement iso2
431431

432432
statement ok
433433
COMMIT
434+
435+
user root
436+
437+
statement ok
438+
BEGIN;
439+
SELECT * FROM t WHERE k = 'a' FOR UPDATE;
440+
441+
user testuser
442+
443+
statement ok
444+
BEGIN;
445+
SET enable_shared_locking_for_serializable = true
446+
447+
statement async share1
448+
SELECT * FROM t WHERE k = 'a' FOR SHARE;
449+
450+
user testuser 2
451+
452+
statement ok
453+
BEGIN;
454+
455+
statement async put1
456+
DELETE FROM t WHERE k = 'a';
457+
458+
user root
459+
460+
# Verify that the waiting transaction reports Shared strength, not Exclusive.
461+
query TTTTTTBB colnames,retry,rowsort
462+
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name = 't'
463+
----
464+
database_name schema_name table_name lock_key_pretty lock_strength durability granted contended
465+
test public t /Table/106/1/"a"/0 Exclusive Unreplicated true true
466+
test public t /Table/106/1/"a"/0 Shared Unreplicated false true
467+
test public t /Table/106/1/"a"/0 Intent Unreplicated false true
468+
469+
statement ok
470+
COMMIT
471+
472+
user testuser
473+
474+
awaitstatement share1
475+
476+
statement ok
477+
COMMIT
478+
479+
user testuser2
480+
481+
awaitstatement put1
482+
483+
statement ok
484+
ROLLBACK

pkg/sql/logictest/testdata/logic_test/cluster_locks_write_buffering

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -291,7 +291,7 @@ query TTTTTTTBB colnames
291291
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r1 AND txn_id='$txn4'
292292
----
293293
database_name schema_name table_name lock_key_pretty lock_strength durability isolation_level granted contended
294-
test public t /Table/106/1/"b"/0 Exclusive Unreplicated SERIALIZABLE false true
294+
test public t /Table/106/1/"b"/0 Intent Unreplicated SERIALIZABLE false true
295295

296296
query TTTTTTTBB colnames,rowsort
297297
SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, isolation_level, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r2 AND txn_id='$txn3'

0 commit comments

Comments
 (0)