-
Notifications
You must be signed in to change notification settings - Fork 4k
kv: add assertion to prevent lock re-acquisition at prior epochs #103492
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
Conversation
nvb
left a comment
There was a problem hiding this 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 r1.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)
pkg/kv/kvserver/concurrency/lock_table.go line 1878 at r1 (raw file):
// 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(
Returning an error here will propagate back up to managerImpl.OnLockAcquired, which will fatal. Is that the intention? We'd probably be able to see that if you extended the new test to the concurrency manager's logic test framework. Given how unreplicated lock acquisition is built as a side-effect of evaluation, we don't currently have a good way to return an error up to the locking requests. Should we just be discarding the acquisition attempt instead?
arulajmani
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/kvserver/concurrency/lock_table.go line 1878 at r1 (raw file):
Returning an error here will propagate back up to
managerImpl.OnLockAcquired, which will fatal. Is that the intention?
No, that was an oversight on my end.
Given how unreplicated lock acquisition is built as a side-effect of evaluation, we don't currently have a good way to return an error up to the locking requests. Should we just be discarding the acquisition attempt instead?
IIUC, like there's two callers for OnLockAcquired -- one in handleReadWriteLocalEvalResult and handleReadOnlyLocalEvalResult. The latter is for unreplicated locks.
How do you feel about modifyingOnLockAcquired to return errors? Then, on the read-only path, we can return the error back up to the client.
We could then either modify OnLockAcquired to fatal when handed an AssertionFailedError (which will happen on the read-write path) or let the error bubble up and fatal in handleReadWriteLocalEvalResult.
Is there some hazard in bubbling errors up, which prompted the signature of OnLockAcquired to begin with, that I'm missing here?
nvb
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)
pkg/kv/kvserver/concurrency/lock_table.go line 1878 at r1 (raw file):
Previously, arulajmani (Arul Ajmani) wrote…
Returning an error here will propagate back up to
managerImpl.OnLockAcquired, which will fatal. Is that the intention?No, that was an oversight on my end.
Given how unreplicated lock acquisition is built as a side-effect of evaluation, we don't currently have a good way to return an error up to the locking requests. Should we just be discarding the acquisition attempt instead?
IIUC, like there's two callers for
OnLockAcquired-- one inhandleReadWriteLocalEvalResultandhandleReadOnlyLocalEvalResult. The latter is for unreplicated locks.How do you feel about modifying
OnLockAcquiredto return errors? Then, on the read-only path, we can return the error back up to the client.We could then either modify
OnLockAcquiredto fatal when handed anAssertionFailedError(which will happen on the read-write path) or let the error bubble up and fatal inhandleReadWriteLocalEvalResult.Is there some hazard in bubbling errors up, which prompted the signature of
OnLockAcquiredto begin with, that I'm missing here?
I think that makes sense if we are able to ensure that handleReadWriteLocalEvalResult is never called for unreplicated lock acquisition. Can we make that guarantee? To make it, we'd need to ensure that unreplicated lock acquisition and replicated lock acquisition never end up in the same batch, otherwise, we'd hit the read-write path.
My weak suggestion is that this isn't a fight to pick right now and that a silent no-op in these cases is good enough while everything else around here is in flux. But I'd also support digging deeper into this if you think it's important.
arulajmani
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
pkg/kv/kvserver/concurrency/lock_table.go line 1878 at r1 (raw file):
To make it, we'd need to ensure that unreplicated lock acquisition and replicated lock acquisition never end up in the same batch, otherwise, we'd hit the read-write path.
I was thinking we could, because the DistSender splits out read batches:
cockroach/pkg/kv/kvpb/batch.go
Lines 701 to 706 in b19a4a8
| // NOTE: One reason for splitting reads from writes is that write-only batches | |
| // can sometimes have their read timestamp bumped on the server, which doesn't | |
| // work for read requests due to how the timestamp-aware latching works (i.e. a | |
| // read that acquired a latch @ ts10 can't simply be bumped to ts 20 because | |
| // there might have been overlapping writes in the 10..20 window). | |
| func (ba BatchRequest) Split(canSplitET bool) [][]RequestUnion { |
Specifically, this means that Get, RevScan, and Scan batches should never go through the read-write path, and these are the only requests that acquire unreplicated locks.
I may be missing something. Or, if you tell me this isn't something we should be relying on, I'd buy that too.
Lock re-acquisitions from requests that belong to prior epochs is not allowed. For replicated locks, mvccPutInternal rejects such requests with an error. However, for unreplicated locks where the in-memory lock table is the source of truth, no such check happens. This patch adds one. The comment on `lockHolderInfo.txn` field suggests this was always the intended behavior; it was just never enforced. While here, we also add an assertion to ensure a replicated lock acquisition never calls into the lock table with a transaction belonging to a prior epoch. We expect this case to be handled in mvccPutInternal, so unlike the unreplicated case, this deserves to be an assertion failure. This patch also updates a test that unintentionally re-acquired a lock using a txn from an older epoch. We also add a new test. Informs: cockroachdb#102269 Release note: None
03a7cf9 to
b0a264f
Compare
arulajmani
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Switched things to be in line with our conversation yesterday. PTAL.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten)
nvb
left a comment
There was a problem hiding this 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 r2, all commit messages.
Reviewable status:complete! 1 of 0 LGTMs obtained (waiting on @arulajmani)
|
TFTR! bors r=nvanbenschoten |
|
Build succeeded: |
Lock re-acquisitions from requests that belong to prior epochs is not allowed. For replicated locks, mvccPutInternal rejects such requests with an error. However, for unreplicated locks where the in-memory lock table is the source of truth, no such check happens. This patch adds one. The comment on
lockHolderInfo.txnfield suggests this was always the intended behavior; it was just never enforced.While here, we also add an assertion to ensure a replicated lock acquisition never calls into the lock table with a transaction belonging to a prior epoch. We expect this case to be handled in mvccPutInternal, so unlike the unreplicated case, this deserves to be an assertion failure.
This patch also updates a test that unintentionally re-acquired a lock using a txn from an older epoch. We also add a new test.
Informs: #102269
Release note: None