-
Notifications
You must be signed in to change notification settings - Fork 3.9k
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
rangefeed: fix handling of aborted transactions found by txnPushAttempt #35889
rangefeed: fix handling of aborted transactions found by txnPushAttempt #35889
Conversation
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.
@tbg and/or @bdarnell can I get your eyes on this, too? I'm not quite the careful reviewer that the two of you are and it seems like this is all more complicated than anyone initially thought
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @danhhz and @nvanbenschoten)
pkg/storage/engine/enginepb/mvcc3.proto, line 247 at r1 (raw file):
MVCCCommitIntentOp commit_intent = 4; MVCCAbortIntentOp abort_intent = 5; MVCCAbortTxnOp abort_txn = 6;
What happens if one of these gets sent during a rolling upgrade to a node that doesn't have this branch in the if
? Looks like a panic?
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 @bdarnell, @danhhz, and @tbg)
pkg/storage/engine/enginepb/mvcc3.proto, line 247 at r1 (raw file):
Previously, danhhz (Daniel Harrison) wrote…
What happens if one of these gets sent during a rolling upgrade to a node that doesn't have this branch in the
if
? Looks like a panic?
We only ever send them to our local rangefeed from txnPushAttempt
tasks, so that's not a concern.
@@ -414,6 +462,24 @@ func (uiq *unresolvedIntentQueue) updateTxn( | |||
return false | |||
} | |||
|
|||
// Del removes the transaction from the queue. It returns whether the update had |
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.
This method is exactly what was removed in https://github.com/cockroachdb/cockroach/pull/35777/files#diff-425bf0ccd6243bd7bb88910fc308bdf4L424.
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.
What's the reasoning behind clamping the refcounts to zero? If we let them go negative, wouldn't they either go back to zero or the txn would be deleted wholesale thanks to having been aborted? I'm asking because a) that seems conceptually simpler and b) perhaps we could retain some assertions? For example, if the queue is init'ed and we're seeing a negative refcount, then either it goes away organically or we eventually find that the txn is aborted. In particular, a "stuck" negative refcount for which we discover a committed txn would trigger an error. Maybe this is is also possible with the clamping, not sure. I don't understand the significance of the value zero. I thought once a refcount of zero was reached, the txn would be deleted from the queue, but I don't think that's happening here.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @bdarnell, @nvanbenschoten, and @tbg)
pkg/storage/engine/enginepb/mvcc3.proto, line 219 at r1 (raw file):
instance
pkg/storage/rangefeed/processor_test.go, line 667 at r1 (raw file):
p.ConsumeLogicalOps( writeIntentOpWithKey(txn1Meta.ID, txn1Meta.Key, txn1Meta.Timestamp), writeIntentOpWithKey(txn2Meta.ID, txn2Meta.Key, txn2Meta.Timestamp),
What's this change about?
9628a32
to
145d156
Compare
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.
What's the reasoning behind clamping the refcounts to zero?
I thought once a refcount of zero was reached, the txn would be deleted from the queue
That is what's happening here. We never store ref counts of zero in the queue. We always remove it in that case. You can think of a MVCCAbortIntentOp
as causing a -1
refcount delta and a MVCCAbortTxnOp
as a causing a -count
refcount delta. But notice that we're now overcounting intents, so we need to clamp at zero. This is ok though, because we know that we'll only ever be saturating at zero for aborted transactions.
I think you might be suggesting that we let ref counts go negative when we overcount and then rely on a second MVCCAbortTxnOp
to bring the count back to zero. That wouldn't work because it could result in a txn being added back into the queue with a count of -1 after a MVCCAbortIntentOp
and causing a resolved timestamp regression.
Did that answer your question?
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @bdarnell and @tbg)
pkg/storage/engine/enginepb/mvcc3.proto, line 219 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
instance
Done.
pkg/storage/rangefeed/processor_test.go, line 667 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
What's this change about?
This test was erroneously passing before this change because it only had a single intent for the transaction that was eventually discovered as aborted. This allowed the single ref count decrement to function as a full deletion. With this change, the test now only passes when an aborted txn results in a MVCCAbortTxnOp
instead of a MVCCAbortIntentOp
.
Also for what it's worth, this is how this all worked up until #35777. The logic here isn't really new, it's just being adapted to work with that PR, which realized that aborted intents don't always indicate aborted transactions. |
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.
Yep, thanks
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @bdarnell, @nvanbenschoten, and @tbg)
pkg/storage/rangefeed/resolved_timestamp.go, line 454 at r1 (raw file):
} else if txn.refCount < 0 && !uiq.allowNegRefCount { // Ignore txn. return false
Already taking into account that the callers only use deltas of 0, 1, -1 (I had to go look that up because it's not obvious, so perhaps the code should be written more robustly, with larger negative deltas it seems that more oddities come up incl negative final refcounts) so it looks like would only be able to hit the branch if txn.refCount
before the delta
was 0
but then how is it in the queue? The way I picture this is
- txn is aborted and deleted from queue
- txn writes intent a, gets a refcount of one
- txn intent gets aborted, refcount -1 hits the deletion branch
or
- txn aborted and deleted
- txn intent gets aborted (hits the !ok branch above)
so what example hits this code? (might want to add it as a comment while you're at it)
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 9 of 11 files at r1, 2 of 2 files at r2.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @bdarnell and @nvanbenschoten)
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 @bdarnell and @tbg)
pkg/storage/rangefeed/resolved_timestamp.go, line 454 at r1 (raw file):
Previously, tbg (Tobias Grieger) wrote…
Already taking into account that the callers only use deltas of 0, 1, -1 (I had to go look that up because it's not obvious, so perhaps the code should be written more robustly, with larger negative deltas it seems that more oddities come up incl negative final refcounts) so it looks like would only be able to hit the branch if
txn.refCount
before thedelta
was0
but then how is it in the queue? The way I picture this is
- txn is aborted and deleted from queue
- txn writes intent a, gets a refcount of one
- txn intent gets aborted, refcount -1 hits the deletion branch
or
- txn aborted and deleted
- txn intent gets aborted (hits the !ok branch above)
so what example hits this code? (might want to add it as a comment while you're at it)
Yeah this case wasn't being exercised by the external interface of the type, but I included it for robustness. You're completely right though that it's not actually what we want. Instead, we want it to be collapsed in with the previous case like:
if txn.refCount == 0 || (txn.refCount < 0 && !uiq.allowNegRefCount) {
pkg/storage/rangefeed/resolved_timestamp.go, line 467 at r2 (raw file):
// Del removes the transaction from the queue. It returns whether the update had // an effect on the oldest transaction in the queue. func (uiq *unresolvedIntentQueue) Del(txnID uuid.UUID) bool {
@tbg I asked on slack, but I'll keep the conversation on here. Would you find the following more clear?
func (uiq *unresolvedIntentQueue) Del(txnID uuid.UUID) bool {
if uiq.allowNegRefCount {
panic("cannot Del while allowNegRefCount")
}
return uiq.updateTxn(txnID, nil, hlc.Timestamp{}, math.MinInt64)
}
To avoid underflow, we'd need the allowNegRefCount
check, which is kind of gross.
Fixes cockroachdb#35816. The fix in cockroachdb#35777 inadvertently broke `txnPushAttempt` and its interaction with transactions that were found to be aborted. This commit restores this handling by reverting part of the assertion added in cockroachdb#35772 and splitting up the handling of "aborted intents" from the handling of "aborted transactions". Aborted transactions were always one of the most tricky situations to handle in rangefeed. The challenge is that rangefeed may be tracking a transaction that has been abandoned and may want to push it to unblock its resolved timestamp, but it doesn't have a way to know where the aborted transaction's intents live on its range. The aborted transaction record can't tell us this either. So the only thing rangefeed can do is push the transaction and convince itself that the aborted intents on its range can be safely ignored. This poses a problem for accurate reference counting of intents though. We need to be able to discard a transaction and still handle the eventual resolution of its aborted intents. To handle this properly, the change splits up `MVCCAbortIntentOp` into `MVCCAbortIntentOp` and `MVCCAbortTxnOp`. Before this change, the single operation type was attempting to fill both roles, but they mean fundamentally different things. `MVCCAbortTxnOp` is then given back its ability to delete transactions from the queue directly without worrying about their reference count. Next, the `unresolvedIntentQueue` is extended to be a little more flexible. Instead of asserting that reference counts never drop below zero, it just ignores changes that would make reference counts negative. This has the potential to "leak" reference counts that saturate at zero and are then incremented again, but we know that this will only ever happen for aborted transactions, which can always be pushed again to rediscover that they have been aborted. Release note: None
145d156
to
aa73a72
Compare
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 @bdarnell and @tbg)
pkg/storage/rangefeed/resolved_timestamp.go, line 454 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Yeah this case wasn't being exercised by the external interface of the type, but I included it for robustness. You're completely right though that it's not actually what we want. Instead, we want it to be collapsed in with the previous case like:
if txn.refCount == 0 || (txn.refCount < 0 && !uiq.allowNegRefCount) {
Done.
pkg/storage/rangefeed/resolved_timestamp.go, line 467 at r2 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
@tbg I asked on slack, but I'll keep the conversation on here. Would you find the following more clear?
func (uiq *unresolvedIntentQueue) Del(txnID uuid.UUID) bool { if uiq.allowNegRefCount { panic("cannot Del while allowNegRefCount") } return uiq.updateTxn(txnID, nil, hlc.Timestamp{}, math.MinInt64) }
To avoid underflow, we'd need the
allowNegRefCount
check, which is kind of gross.
Ok, I added this as a comment instead.
@tbg I'm not sure whether you were planning on giving this another look or not. I'm hoping to get it in for tonight's roachtests. |
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.
LGTM, although I'm not very familiar with this code.
Reviewed 8 of 11 files at r1, 2 of 2 files at r2, 1 of 1 files at r3.
Reviewable status: complete! 0 of 0 LGTMs obtained (waiting on @tbg)
TFTRs! bors r+ |
Merge it, I'll take another pass tomorrow but have no reason to assume
there's anything I'd want to have changed
|
35889: rangefeed: fix handling of aborted transactions found by txnPushAttempt r=nvanbenschoten a=nvanbenschoten Fixes #35816. The fix in #35777 inadvertently broke `txnPushAttempt` and its interaction with transactions that were found to be aborted. This commit restores this handling by reverting part of the assertion added in #35772 and splitting up the handling of "aborted intents" from the handling of "aborted transactions". Aborted transactions were always one of the most tricky situations to handle in rangefeed. The challenge is that rangefeed may be tracking a transaction that has been abandoned and may want to push it to unblock its resolved timestamp, but it doesn't have a way to know where the aborted transaction's intents live on its range. The aborted transaction record can't tell us this either. So the only thing rangefeed can do is push the transaction and convince itself that the aborted intents on its range can be safely ignored. This poses a problem for accurate reference counting of intents though. We need to be able to discard a transaction and still handle the eventual resolution of its aborted intents. To handle this properly, the change splits up `MVCCAbortIntentOp` into `MVCCAbortIntentOp` and `MVCCAbortTxnOp`. Before this change, the single operation type was attempting to fill both roles, but they mean fundamentally different things. `MVCCAbortTxnOp` is then given back its ability to delete transactions from the queue directly without worrying about their reference count. Next, the `unresolvedIntentQueue` is extended to be a little more flexible. Instead of asserting that reference counts never drop below zero, it just ignores changes that would make reference counts negative. This has the potential to "leak" reference counts that saturate at zero and are then incremented again, but we know that this will only ever happen for aborted transactions, which can always be pushed again to rediscover that they have been aborted. Release note: None Co-authored-by: Nathan VanBenschoten <[email protected]>
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 1 of 1 files at r3.
Reviewable status: complete! 0 of 0 LGTMs obtained
Build succeeded |
Fixes #35816.
The fix in #35777 inadvertently broke
txnPushAttempt
and its interaction withtransactions that were found to be aborted. This commit restores this handling
by reverting part of the assertion added in #35772 and splitting up the handling
of "aborted intents" from the handling of "aborted transactions".
Aborted transactions were always one of the most tricky situations to handle in
rangefeed. The challenge is that rangefeed may be tracking a transaction that
has been abandoned and may want to push it to unblock its resolved timestamp,
but it doesn't have a way to know where the aborted transaction's intents live
on its range. The aborted transaction record can't tell us this either. So the
only thing rangefeed can do is push the transaction and convince itself that the
aborted intents on its range can be safely ignored. This poses a problem for
accurate reference counting of intents though. We need to be able to discard a
transaction and still handle the eventual resolution of its aborted intents.
To handle this properly, the change splits up
MVCCAbortIntentOp
intoMVCCAbortIntentOp
andMVCCAbortTxnOp
. Before this change, the singleoperation type was attempting to fill both roles, but they mean fundamentally
different things.
MVCCAbortTxnOp
is then given back its ability to deletetransactions from the queue directly without worrying about their reference
count. Next, the
unresolvedIntentQueue
is extended to be a little moreflexible. Instead of asserting that reference counts never drop below zero, it just
ignores changes that would make reference counts negative. This has the potential
to "leak" reference counts that saturate at zero and are then incremented again,
but we know that this will only ever happen for aborted transactions, which can
always be pushed again to rediscover that they have been aborted.
Release note: None