concurrency: do not clear lock holders in tryActiveWait#104782
Conversation
1c69fcc to
6e97a4e
Compare
nvb
left a comment
There was a problem hiding this comment.
Reviewed 1 of 3 files at r1, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)
pkg/kv/kvserver/concurrency/lock_table.go line 492 at r1 (raw file):
// on locks belonging to finalized transactions, we wouldn't need to bother // scanning requests. finalizedUnreplicatedLocks []roachpb.LockUpdate
Is there a better way to draw a parallel between these two slices? Maybe by calling this one toResolveUnreplicated or something like that?
pkg/kv/kvserver/concurrency/lock_table.go line 754 at r1 (raw file):
// before returning. Note that these are only unreplicated locks. Replicated // locks are handled via the g.toResolve. var locksToGC []*lockState
Reminder that we can now remove this.
pkg/kv/kvserver/concurrency/lock_table.go line 762 at r1 (raw file):
// finalized transactions. We do so regardless of whether this request can // proceed to evaluation or needs to wait at some conflicting lock. if len(g.finalizedUnreplicatedLocks) > 0 {
We'll need to clear this slice after resolving the locks. A method like the following might help make the behavior clearer and less error prone:
func (g *lockTableGuardImpl) TakeResolveUnreplicated() []roachpb.LockUpdate {
r := g.toResolveUnreplicated
g.toResolveUnreplicated = nil
return r
}
...
defer func() {
// Eagerly update any unreplicated locks that are known to belong to
// finalized transactions. We do so regardless of whether this request can
// proceed to evaluation or needs to wait at some conflicting lock.
if r := g.TakeResolveUnreplicated(); len(r) > 0 {
for i := range r {
g.lt.updateLockInternal(&r[i])
}
}
}()pkg/kv/kvserver/concurrency/lock_table.go line 1838 at r1 (raw file):
waitForState.queuedWriters = l.queuedWriters.Len() // update field } if (replicatedLockFinalizedTxn != nil || !l.holder.locked) && l.queuedWriters.Front().Value.(*queuedGuard) == qg {
Do we need a case similar to this (e.g. with a unreplicatedLockFinalizedTxn variable) so that locking request don't wait and terminate their scan?
If I understand correctly, this logic will cause a locking request that stumbles onto an uncontended, unreplicated lock with a finalized lock holder to initially wait without a claim and stop scanning in resumeScan. It will then call updateLockInternal and be given the claim, followed by a call to doneActivelyWaitingAtLock. But by this point, it will need to drop its latches and enter the lockTableWaiter to complete the rest of its scan. That's a regression compared to today's behavior, where it will not terminate its scan in these cases (for replicated or unreplicated locks with finalized lock holders), right?
pkg/kv/kvserver/concurrency/lock_table.go line 1845 at r1 (raw file):
} } else { if replicatedLockFinalizedTxn != nil {
Similarly, should we extend this logic instead of adding the early return above ("Non-locking reads do not need to wait on a finalized unreplicated locks")?
6e97a4e to
924e50d
Compare
arulajmani
left a comment
There was a problem hiding this comment.
TFTR, RFAL!
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/kvserver/concurrency/lock_table.go line 492 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Is there a better way to draw a parallel between these two slices? Maybe by calling this one
toResolveUnreplicatedor something like that?
Done.
pkg/kv/kvserver/concurrency/lock_table.go line 754 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Reminder that we can now remove this.
Done.
pkg/kv/kvserver/concurrency/lock_table.go line 762 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
We'll need to clear this slice after resolving the locks. A method like the following might help make the behavior clearer and less error prone:
func (g *lockTableGuardImpl) TakeResolveUnreplicated() []roachpb.LockUpdate { r := g.toResolveUnreplicated g.toResolveUnreplicated = nil return r } ... defer func() { // Eagerly update any unreplicated locks that are known to belong to // finalized transactions. We do so regardless of whether this request can // proceed to evaluation or needs to wait at some conflicting lock. if r := g.TakeResolveUnreplicated(); len(r) > 0 { for i := range r { g.lt.updateLockInternal(&r[i]) } } }()
Good call. I don't think this needs to be public or an interface method, so I've kept it private for now. Let me know if you feel otherwise.
pkg/kv/kvserver/concurrency/lock_table.go line 1838 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Do we need a case similar to this (e.g. with a
unreplicatedLockFinalizedTxnvariable) so that locking request don'twaitand terminate their scan?If I understand correctly, this logic will cause a locking request that stumbles onto an uncontended, unreplicated lock with a finalized lock holder to initially wait without a claim and stop scanning in
resumeScan. It will then callupdateLockInternaland be given the claim, followed by a call todoneActivelyWaitingAtLock. But by this point, it will need to drop its latches and enter the lockTableWaiter to complete the rest of its scan. That's a regression compared to today's behavior, where it will not terminate its scan in these cases (for replicated or unreplicated locks with finalized lock holders), right?
Nice catch! I should have double checked, but I thought this was "just working" because of this test change https://github.com/cockroachdb/cockroach/pull/104782/files#diff-1c7339babc1de5ff85a0d7e99a5eaa42f625540090fae3a0edff4b85a0aac47fL353-R360 -- the waiter is in the correct (inactive) state. But after staring at this a bit more, that's because we're updating the lock one layer above, which is where it's being marked as inactive.
Anyway, I added some more testing here. PTAL.
pkg/kv/kvserver/concurrency/lock_table.go line 1845 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Similarly, should we extend this logic instead of adding the early return above ("Non-locking reads do not need to wait on a finalized unreplicated locks")?
Done.
|
@arulajmani let me know when this is rebased on master and I'll give it one last pass. |
|
I forgot there were conflicts here with your PR that landed. Thanks for the nudge! |
924e50d to
88afedf
Compare
|
@nvanbenschoten rebased, PTAL. |
nvb
left a comment
There was a problem hiding this comment.
Reviewed 2 of 3 files at r3, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @arulajmani)
pkg/kv/kvserver/concurrency/lock_table.go line 784 at r3 (raw file):
// proceed to evaluation or needs to wait at some conflicting lock. // // Note that replicated locks are handled differently, using the g.ToResolve
nit: g.toResolve
pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks line 829 at r3 (raw file):
# Non-locking, transactional request new-request r=req16 txn=none ts=11,0 spans=intent@c
Should we make each of these requests scan over multiple (two) unreplicated, known-finalized locks? The doneWaiting here shows that none of these are waiting after encountering the locks (the bug from before), but resolving multiple unreplicated locks might prove a little more.
Previous to this patch, tryActiveWait would clear a lock's holder if it belonged to a finalized transaction and was only held with unreplicated durability. It would also nudge some request's in the lock's wait queue to proceed. This state transition inside of tryActiveWait for other requests was subtle. Arguably, the function which decides whether to wait on a lock or not should not be responsible for performing such state transitions for other requests. This patch slightly improves this situation. We no longer return a transitionedToFree boolean and expect the caller to take some action based on it. Instead, we accumulate unreplicated locks that belong to finalized transactions in the request guard, as it sequences. The request then assumes the responsibility for clearing such locks and nudging waiters (if possible). This last part is still not great. However, we need it for practical purposes. In the ideal world, we would address the TODO in TransactionIsFinalized, and perform this state transition there. But until then, this patch moves the needle slightly. More importantly, it allows us to remove some special casing when tryActiveWait is removed and replaced, over in cockroachdb#104620. In particular, locking requests that come across unreplicated locks that belong to finalized transactions and have empty wait queues will be able to acquire claims on such locks before proceeding. This can be seen in the test diff for `clear_finalized_txn_locks`. Release note: None
88afedf to
85f99fd
Compare
arulajmani
left a comment
There was a problem hiding this comment.
TFTR!
bors r=nvanbenschoten
Reviewable status:
complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @nvanbenschoten)
pkg/kv/kvserver/concurrency/testdata/lock_table/clear_finalized_txn_locks line 829 at r3 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Should we make each of these requests scan over multiple (two) unreplicated, known-finalized locks? The
doneWaitinghere shows that none of these are waiting after encountering the locks (the bug from before), but resolving multiple unreplicated locks might prove a little more.
Done.
|
Build succeeded: |
Previous to this patch, tryActiveWait would clear a lock's holder if it belonged to a finalized transaction and was only held with unreplicated durability. It would also nudge some request's in the lock's wait queue to proceed. This state transition inside of tryActiveWait for other requests was subtle. Arguably, the function which decides whether to wait on a lock or not should not be responsible for performing such state transitions for other requests.
This patch slightly improves this situation. We no longer return a transitionedToFree boolean and expect the caller to take some action based on it. Instead, we accumulate unreplicated locks that belong to finalized transactions in the request guard, as it sequences. The request then assumes the responsibility for clearing such locks and nudging waiters (if possible).
This last part is still not great. However, we need it for practical purposes. In the ideal world, we would address the TODO in TransactionIsFinalized, and perform this state transition there. But until then, this patch moves the needle slightly. More importantly, it allows us to remove some special casing when tryActiveWait is removed and replaced, over in #104620. In particular, locking requests that come across unreplicated locks that belong to finalized transactions and have empty wait queues will be able to acquire claims on such locks before proceeding. This can be seen in the test diff for
clear_finalized_txn_locks.Release note: None