Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

concurrency: implement generalized lock promotion #119671

Merged
merged 1 commit into from Mar 22, 2024

Conversation

arulajmani
Copy link
Collaborator

@arulajmani arulajmani commented Feb 27, 2024

Previously, if a request had acquired a shared lock, it wasn't able to
promote it to an Exclusive or Intent (by writing to the same key) lock.
This was because the lock table could not detect deadlock scenarios
where two transactions that both held shared locks were trying to
promote. Moreover, it also couldn't detect wait queue local deadlocks
that involved non-transactional requests.

These two limitations have now been limited. For the former, we're able
to leverage our existing deadlock detection algorithm by performing the
correct set of pushes. This is done by changing the claimantTxn concept
slightly. Previously, there could only be one claimant for a key. This
is no longer true -- now, the claimant may be different, depending on
who is asking for it.

For the latter, we reorder the wait queue to avoid preventable
deadlocks. This is done by preferring lock promoters over other
requests. The bulk of this was already done in
#118484.

Closes #110435

Release note: shared locks can now be re-acquired with higher strength.

@arulajmani arulajmani requested a review from a team as a code owner February 27, 2024 01:08
Copy link

blathers-crl bot commented Feb 27, 2024

Your pull request contains more than 1000 changes. It is strongly encouraged to split big PRs into smaller chunks.

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

@cockroach-teamcity
Copy link
Member

This change is Reviewable

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 1 of 4 files at r3, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)


-- commits line 51 at r3:
Does this need a category?


pkg/kv/kvserver/concurrency/lock_table.go line 2432 at r2 (raw file):

) error {
	if held == lock.Shared && reAcquisitionStr > held {
		return MarkLockPromotionError(errors.UnimplementedErrorf(

Can we now remove the following?

  • MarkLockPromotionError
  • LockPromotionError
  • exceptSharedLockPromotionError

pkg/kv/kvserver/concurrency/lock_table.go line 82 at r3 (raw file):

	// by having r3 wait in the waitSelf state and not push txn2. If r1 acquires the lock,
	// r3 will be re-ordered to proceed before r2; if it doesn't, r2 no longer depends
	// on txn1 and can thus proceed. Either way, the deadlock is avoided.

Referring to this as a "deadlock" that's avoided by "re-ordering" is kind of confusing, at least to me. There's no txn deadlock nor re-ordering here, is there? Additionally, r3 would never push r2 (everyone pushes the head), so there's not even a risk of a false positive deadlock.

I'm glad we have the example here, but maybe we adjust the wording around it a bit?


pkg/kv/kvserver/concurrency/lock_table.go line 2087 at r3 (raw file):

// specifics from the lock table waiter (which is responsible for pushing).
//
// There is only a single claimant transaction for a key from a request's

Given that "claimant txn" is no longer a global property of a key, would it make sense to rename this method to claimantTxnFor?


pkg/kv/kvserver/concurrency/lock_table.go line 2114 at r3 (raw file):

		// key.
		for e := kl.holders.Front(); e != nil; e = e.Next() {
			if !g.isSameTxn(e.Value.txn) {

I recently thought through this, but forget where I landed — what will happen if a request encounters a key with two lock holders, both from other txns, and determines based on the finalizedTxns cache that the first has already been aborted. Will it consider the second to be the claimantTxn, or the first? If the first, will it end up pushing someone that it knows is finalized? Will that cause it to get stuck? Or will it just undermine deferred intent resolution (toResolve)?


pkg/kv/kvserver/concurrency/lock_table.go line 2123 at r3 (raw file):

	}
	qg := kl.queuedLockingRequests.Front().Value
	return qg.guard.txnMeta(), false

Consider adding a comment that this can be the same txn as g, and that's ok. Then maybe reference waitSelf.


pkg/kv/kvserver/concurrency/lock_table.go line 2427 at r3 (raw file):

//
// REQUIRES: kl.mu to be locked.
// TODO(arul): We can get rid of the error path here.

👍 was going to leave a comment. You beat me to it.


pkg/kv/kvserver/concurrency/lock_table.go line 2654 at r3 (raw file):

func (kl *keyLocks) insertLockingRequest(
	g *lockTableGuardImpl, accessStrength lock.Strength,
) (*queuedGuard, error) {

We can get rid of this error path as well, right?


pkg/kv/kvserver/concurrency/lock_table.go line 3140 at r3 (raw file):

			// requests will be able to promote their locks -- otherwise, we'd be
			// cancelling transaction pushes for no good reason.
			if kl.holders.Len() == 1 {

Does this have the same effect as recomputeWaitQueues?


pkg/kv/kvserver/concurrency/lock_table.go line 3140 at r3 (raw file):

			// requests will be able to promote their locks -- otherwise, we'd be
			// cancelling transaction pushes for no good reason.
			if kl.holders.Len() == 1 {

Do we need this same logic below on the if !isLocked { path?


pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked line 364 at r3 (raw file):

locked: false

# TODO(arul): add a test where there are 2 shared locks and one of them is trying

Want to address this TODO in the same PR, since it's for more testing?

Copy link
Collaborator Author

@arulajmani arulajmani left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should be ready for another look!

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)


-- commits line 51 at r3:

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Does this need a category?

Yeah, done.


pkg/kv/kvserver/concurrency/lock_table.go line 2432 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Can we now remove the following?

  • MarkLockPromotionError
  • LockPromotionError
  • exceptSharedLockPromotionError

Yeah, we should be able to do this now. I was planning on doing it in a followup and leaving the TODOs in for now.


pkg/kv/kvserver/concurrency/lock_table.go line 82 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Referring to this as a "deadlock" that's avoided by "re-ordering" is kind of confusing, at least to me. There's no txn deadlock nor re-ordering here, is there? Additionally, r3 would never push r2 (everyone pushes the head), so there's not even a risk of a false positive deadlock.

I'm glad we have the example here, but maybe we adjust the wording around it a bit?

I reworded this to use the words "dependency cycle" instead and tried to elaborate what I mean by the "re-ordering". Let me know if this is any better.


pkg/kv/kvserver/concurrency/lock_table.go line 2087 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Given that "claimant txn" is no longer a global property of a key, would it make sense to rename this method to claimantTxnFor?

Good point, done.


pkg/kv/kvserver/concurrency/lock_table.go line 2114 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

I recently thought through this, but forget where I landed — what will happen if a request encounters a key with two lock holders, both from other txns, and determines based on the finalizedTxns cache that the first has already been aborted. Will it consider the second to be the claimantTxn, or the first? If the first, will it end up pushing someone that it knows is finalized? Will that cause it to get stuck? Or will it just undermine deferred intent resolution (toResolve)?

You're right that the first one will be considered the claimant here. I traced through this, and we'll end up undermining deferred intent resolution as we'll immediately resolve the intent once the push returns after pushing the finalized transaction.


pkg/kv/kvserver/concurrency/lock_table.go line 2123 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Consider adding a comment that this can be the same txn as g, and that's ok. Then maybe reference waitSelf.

Done.


pkg/kv/kvserver/concurrency/lock_table.go line 2654 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We can get rid of this error path as well, right?

Yeah, we should be able to. If okay by you, let's do all the error path cleanup in a quick followup PR?


pkg/kv/kvserver/concurrency/lock_table.go line 3140 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Does this have the same effect as recomputeWaitQueues?

It doesn't, because recomputeWaitQueues doesn't take lock holder(s) transaction into account when recomputing wait queues. So if we were to call recomputeWaitQueues here when the lock looked something like:

[key a] - [txn1 shared]

  • wait-queue: [r1 txn1 exclusive]

It wouldn't remove r1 because of:

if lock.Conflicts(qlr.mode, strongestMode, &st.SV) {
break
}


pkg/kv/kvserver/concurrency/lock_table.go line 3140 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Do we need this same logic below on the if !isLocked { path?

Good catch. Done, and added a test as well.


pkg/kv/kvserver/concurrency/testdata/lock_table/skip_locked line 364 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Want to address this TODO in the same PR, since it's for more testing?

Done.

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 1 of 4 files at r3, 7 of 52 files at r4, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)


pkg/kv/kvserver/concurrency/lock_table.go line 2432 at r2 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

Yeah, we should be able to do this now. I was planning on doing it in a followup and leaving the TODOs in for now.

👍


pkg/kv/kvserver/concurrency/lock_table.go line 82 at r3 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

I reworded this to use the words "dependency cycle" instead and tried to elaborate what I mean by the "re-ordering". Let me know if this is any better.

This sounds better. The only thing you might want to do it make it clear that the "re-ordering" takes place after the first request acquires the lock. So the waitSelf state is just a way to have r3 sit tight until r1 has acquired the lock and triggers the re-ordering.


pkg/kv/kvserver/concurrency/lock_table.go line 2114 at r3 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

You're right that the first one will be considered the claimant here. I traced through this, and we'll end up undermining deferred intent resolution as we'll immediately resolve the intent once the push returns after pushing the finalized transaction.

👍 thanks for confirming. That's unfortunate, but I can't think of a case where it's terribly problematic today. We'll just need to address it if we want to address #94730 in the way we've talked about.


pkg/kv/kvserver/concurrency/lock_table.go line 2654 at r3 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

Yeah, we should be able to. If okay by you, let's do all the error path cleanup in a quick followup PR?

👍


pkg/kv/kvserver/concurrency/lock_table.go line 88 at r4 (raw file):

	// head of the queue everyone is pushing belongs to its own transaction, it
	// knows there is a local dependency cycle.
	// Without any re-ordering, this is a guaranteed deadlock, as r2 from txn2 is

This sentence seems to be cut off.


pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion line 236 at r4 (raw file):

----
start-waiting: false

Want to print here, before the acquire?


pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion line 314 at r4 (raw file):

num=1
 lock: "a"
  holder: txn: 00000000-0000-0000-0000-000000000001 epoch: 0, iso: Serializable, info: unrepl [(str: Shared seq: 0)]

As written with the call to releaseLockingRequestsFromTxn, we're removing the promoter from the wait-queue. Is that consistent with the first new test case here, where the promoter becomes an inactive waiter? Should we be marking the promoter as inactive instead of removing it?

EDIT: the next test case reminded me of how this is supposed to work. We release the request and let it re-enqueue as an inactive waiter. I'll ask you in person why we do it this way.

Copy link
Collaborator Author

@arulajmani arulajmani left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTR, RFAL

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)


pkg/kv/kvserver/concurrency/lock_table.go line 82 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

This sounds better. The only thing you might want to do it make it clear that the "re-ordering" takes place after the first request acquires the lock. So the waitSelf state is just a way to have r3 sit tight until r1 has acquired the lock and triggers the re-ordering.

Done.


pkg/kv/kvserver/concurrency/lock_table.go line 2114 at r3 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

👍 thanks for confirming. That's unfortunate, but I can't think of a case where it's terribly problematic today. We'll just need to address it if we want to address #94730 in the way we've talked about.

SGTM


pkg/kv/kvserver/concurrency/lock_table.go line 88 at r4 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

This sentence seems to be cut off.

This was meant to be deleted.


pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion line 236 at r4 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Want to print here, before the acquire?

Done.


pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion line 314 at r4 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

As written with the call to releaseLockingRequestsFromTxn, we're removing the promoter from the wait-queue. Is that consistent with the first new test case here, where the promoter becomes an inactive waiter? Should we be marking the promoter as inactive instead of removing it?

EDIT: the next test case reminded me of how this is supposed to work. We release the request and let it re-enqueue as an inactive waiter. I'll ask you in person why we do it this way.

We spoke about this offline and decided to make recomputeWaitQueues smarter. It now recognizes the case where a lock is being promoted, and is able to mark the request trying to promote its lock as an inactive waiter instead of removing it.

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 2 of 2 files at r5, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)


pkg/kv/kvserver/concurrency/lock_table.go line 3169 at r5 (raw file):

			// to informActiveWaiters will compute who this guy should be and let the
			// waiters know.
			kl.informActiveWaiters()

recomputeWaitQueues calls informActiveWaiters, so this can go in an else branch. Same thing below.

Also, consider rewording some of this to make it clear that the != 1 case is the optimization, not the other way around. We could just always call recomputeWaitQueues, but we chose not to because we know the wait-queue won't change if there are still 2 or more lock holders.


pkg/kv/kvserver/concurrency/lock_table.go line 3300 at r5 (raw file):

It's the caller's responsibility to not needlessly call into this method.

I feel like this sentence isn't needed. The caller is free to call into this method, it just won't always do something. That's true in plenty of other cases as well.

This relates to my comment about about making it clear that not calling into recomputeWaitQueues in certain cases is the optimization. And being an optimization, it does not affect behavior, just performance.


pkg/kv/kvserver/concurrency/lock_table.go line 3342 at r5 (raw file):

		curr := e
		e = e.Next()
		if belongsToOnlyLockHolder(reader) || !lock.Conflicts(reader.curLockMode(), strongestMode, &st.SV) {

We have this same condition twice, except its negated below. Want to pull it out into a closure?

Copy link
Collaborator Author

@arulajmani arulajmani left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)


pkg/kv/kvserver/concurrency/lock_table.go line 3169 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

recomputeWaitQueues calls informActiveWaiters, so this can go in an else branch. Same thing below.

Also, consider rewording some of this to make it clear that the != 1 case is the optimization, not the other way around. We could just always call recomputeWaitQueues, but we chose not to because we know the wait-queue won't change if there are still 2 or more lock holders.

Reworded the commentary here a bit and added an else branch.


pkg/kv/kvserver/concurrency/lock_table.go line 3300 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

It's the caller's responsibility to not needlessly call into this method.

I feel like this sentence isn't needed. The caller is free to call into this method, it just won't always do something. That's true in plenty of other cases as well.

This relates to my comment about about making it clear that not calling into recomputeWaitQueues in certain cases is the optimization. And being an optimization, it does not affect behavior, just performance.

Dropped the line.


pkg/kv/kvserver/concurrency/lock_table.go line 3342 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

We have this same condition twice, except its negated below. Want to pull it out into a closure?

I was initially hesitant because the closure name is a bit verbose, but done.

Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)


pkg/kv/kvserver/concurrency/lock_table.go line 3342 at r5 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

I was initially hesitant because the closure name is a bit verbose, but done.

Actually, do we need to check for this at all in the waitingReaders set? There will never be a waiting reader waiting on a lock that's held by its own txn, because non-locking readers don't conflict with shared locks.

If so, maybe just go back to inlining belongsToOnlyLockHolder in the queuedLockingRequests loop. But consider pulling out a variable for clarity. So something like (with the suggestion from below):

	for e := kl.queuedLockingRequests.Front(); e != nil; {
		qlr := e.Value
		curr := e
		e = e.Next()
		belongsToOnlyLockHolder := lockHolderTxn != nil && g.isSameTxn(lockHolderTxn)
		conflictsWithEarlier := lock.Conflicts(qlr.mode, strongestMode, &st.SV) && !belongsToOnlyLockHolder
		if conflictsWithEarlier {
			// TODO: explain
			if !qlr.order.isPromoting {
				break
			}
			continue
		} 

pkg/kv/kvserver/concurrency/lock_table.go line 3361 at r6 (raw file):

		e = e.Next()
		if !belongsToOnlyLockHolderOrDoesNotConflictsWithStrongestMode(qlr.guard) {
			break

Do we want to break here? Is it possible that we see a promoter that does not belong to the only lock holder first, and then a promoter that does belong to the only lock holder second? Something like:

[key a] - [txn1 shared]
  wait-queue: [r2 txn2 exclusive], [r3 txn1 exclusive]

There seem to be a few different solutions. The simplest is probably to continue until we're through the promoting waiters, and only break once we're to non-promoters. See above.


pkg/kv/kvserver/concurrency/lock_table.go line 3361 at r6 (raw file):

		e = e.Next()
		if !belongsToOnlyLockHolderOrDoesNotConflictsWithStrongestMode(qlr.guard) {
			break

Separate thread to discuss releaseLockingRequestsFromTxn: can we add a comment around the call to releaseLockingRequestsFromTxn to discuss how it interacts with lock promotion? On the surface, it looks like we're getting this wrong and we're allowing a promoter to drop out of the wait-queue as soon as a weaker-strength lock is acquired (even if there are multiple shared locks on the key). It's pretty subtle that the promoter will always scan again and re-enqueue on the same key, this time with an updated queueOrder.

Copy link
Collaborator Author

@arulajmani arulajmani left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)


pkg/kv/kvserver/concurrency/lock_table.go line 3342 at r5 (raw file):

Actually, do we need to check for this at all in the waitingReaders set? There will never be a waiting reader waiting on a lock that's held by its own txn, because non-locking readers don't conflict with shared locks.

Good catch. Done.


pkg/kv/kvserver/concurrency/lock_table.go line 3361 at r6 (raw file):

Is it possible that we see a promoter that does not belong to the only lock holder first, and then a promoter that does belong to the only lock holder second? Something like:

Spoke about this offline, and while this is possible, the solution we had above doesn't fully solve it. From the example, if r3 is marked inactive, it'll re-scan and start waiting again because of r2 -- that's not what we want. Instead, we want to re-order the queue. The new change does this in recomputeWaitQueues.


pkg/kv/kvserver/concurrency/lock_table.go line 3361 at r6 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Separate thread to discuss releaseLockingRequestsFromTxn: can we add a comment around the call to releaseLockingRequestsFromTxn to discuss how it interacts with lock promotion? On the surface, it looks like we're getting this wrong and we're allowing a promoter to drop out of the wait-queue as soon as a weaker-strength lock is acquired (even if there are multiple shared locks on the key). It's pretty subtle that the promoter will always scan again and re-enqueue on the same key, this time with an updated queueOrder.

Added a comment, and also a TODO to push some of this complexity in recomputeWaitQueues.

@arulajmani arulajmani force-pushed the sl-lock-promo-generalized branch 2 times, most recently from 2b3f956 to 062d587 Compare March 21, 2024 18:28
Copy link
Member

@nvanbenschoten nvanbenschoten left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm:

Reviewed 1 of 1 files at r6, 2 of 2 files at r8, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @arulajmani)


pkg/kv/kvserver/concurrency/lock_table.go line 3161 at r8 (raw file):

			gc = kl.releaseWaitersOnKeyUnlocked()
		} else {
			// In some rare cases (such as replays), there may be requests belonging

Do you mind opening a GA-blocker issue to add assertions which would have failed if we did not include this logic? This list is https://cockroachlabs.slack.com/archives/DQ7S0F5EF/p1710865789291819.


pkg/kv/kvserver/concurrency/lock_table.go line 3169 at r8 (raw file):

			// this two different ways:
			// 1. Recompute queueOrder.isPromoting for waiting requests that belong
			//to the transaction that just released the lock and reorder the wait

nit: spacing at the beginning of these two lines.


pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion line 1075 at r8 (raw file):

           txn: 00000000-0000-0000-0000-000000000002 epoch: 1, iso: Serializable, info: unrepl [(str: Shared seq: 0)]
   queued locking requests:
    active: true req: 42, strength: Exclusive, txn: 00000000-0000-0000-0000-000000000001

Should we add promoting: true to these lines when the waiter is promoting (in a separate PR)? We could do it only if qg.order.isPromoting and never add promoting: false so that we don't create a large diff.

While there, we could also switch qg.guard.seqNum to qg.order.reqSeqNum in keyLocks.safeFormat, which is the same value but feels a bit more like what the code is trying to represent.

Previously, if a request had acquired a shared lock, it wasn't able to
promote it to an Exclusive or Intent (by writing to the same key) lock.
This was because the lock table could not detect deadlock scenarios
where two transactions that both held shared locks were trying to
promote. Moreover, it also couldn't detect wait queue local deadlocks
that involved non-transactional requests.

These two limitations have now been limited. For the former, we're able
to leverage our existing deadlock detection algorithm by performing the
correct set of pushes. This is done by changing the claimantTxn concept
slightly. Previously, there could only be one claimant for a key. This
is no longer true -- now, the claimant may be different, depending on
who is asking for it.

For the latter, we reorder the wait queue to avoid preventable
deadlocks. This is done by preferring lock promoters over other
requests. The bulk of this was already done in
cockroachdb#118484.

Closes cockroachdb#110435

Release note (sql change): shared locks (acquired using SELECT FOR
SHARE or implicitly by read committed transactions) can now be
re-acquired with higher strength (using SELECT FOR UPDATE or by writing
to the key).
Copy link
Collaborator Author

@arulajmani arulajmani left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTR!

bors r=nvanbenschoten

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten)


pkg/kv/kvserver/concurrency/lock_table.go line 3161 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Do you mind opening a GA-blocker issue to add assertions which would have failed if we did not include this logic? This list is https://cockroachlabs.slack.com/archives/DQ7S0F5EF/p1710865789291819.

Filed #120909


pkg/kv/kvserver/concurrency/testdata/lock_table/lock_promotion line 1075 at r8 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Should we add promoting: true to these lines when the waiter is promoting (in a separate PR)? We could do it only if qg.order.isPromoting and never add promoting: false so that we don't create a large diff.

While there, we could also switch qg.guard.seqNum to qg.order.reqSeqNum in keyLocks.safeFormat, which is the same value but feels a bit more like what the code is trying to represent.

This sounds good to me. I'll send out a followup.

@craig
Copy link
Contributor

craig bot commented Mar 22, 2024

@craig craig bot merged commit 01c8fa0 into cockroachdb:master Mar 22, 2024
21 of 22 checks passed
arulajmani added a commit to arulajmani/cockroach that referenced this pull request Mar 23, 2024
We only do so if a request is promoting its locks; otherwise, the lock
formatting remains the same as before (which is nice becuase it reduces
useless test churn).

Followup from cockroachdb#119671

Release note: None

Epic: none
arulajmani added a commit to arulajmani/cockroach that referenced this pull request Apr 1, 2024
We only do so if a request is promoting its locks; otherwise, the lock
formatting remains the same as before (which is nice becuase it reduces
useless test churn).

Followup from cockroachdb#119671

Release note: None

Epic: none
arulajmani added a commit to arulajmani/cockroach that referenced this pull request Apr 6, 2024
We only do so if a request is promoting its locks; otherwise, the lock
formatting remains the same as before (which is nice becuase it reduces
useless test churn).

Followup from cockroachdb#119671

Release note: None

Epic: none
craig bot pushed a commit that referenced this pull request Apr 6, 2024
120918: concurrency: include promotion information when printing out locks r=nvanbenschoten a=arulajmani

We only do so if a request is promoting its locks; otherwise, the lock formatting remains the same as before (which is nice becuase it reduces useless test churn).

Followup from #119671

Release note: None

Epic: none

Co-authored-by: Arul Ajmani <arulajmani@gmail.com>
blathers-crl bot pushed a commit that referenced this pull request Apr 6, 2024
We only do so if a request is promoting its locks; otherwise, the lock
formatting remains the same as before (which is nice becuase it reduces
useless test churn).

Followup from #119671

Release note: None

Epic: none
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

kv: correctly handle lock promotion from SHARED to EXCLUSIVE/INTENT
3 participants