-
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
kvcoord: prevent concurrent EndTxn requests #65592
kvcoord: prevent concurrent EndTxn requests #65592
Conversation
This is a fairly naïve implementation, there are likely scenarios such as e.g. leaf txns that aren't properly handled here. However, I figured I'd open a PR and get some feedback on it while verifying. |
5254ac4
to
206edd3
Compare
dfccec8
to
98b53dd
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.
On the "prevent concurrent EndTxn
requests" commit, it seems to me that adding a new interceptor for the single purpose of blocking the async EndTxn
sounds complicated. Have you considered other options? I have a couple of thoughts:
txnPipeliner
indeed collects the locks and the in-flight writes after a response is received. But it doesn't have to. I think the reason why it does it after the response and not before sending the request is because it want to handle optimally the cases where a request has a resume span. However, there's already cases where it ignores - if an error is returned, then in assumes that all of the request's span was written to. So it seems to me that we could have the async rollback also be pessimistic like that. So, we could have the pipeliner maintain tentative intent spans while a request is in progress, that get replaced with the real spans if a response is received.- I think the heartbeater is the right place to have the synchronization between the async rollback and the in-flight requests. It has all the state it needs - it knows whether a request is in flight, and it controls the async rollback.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @aliher1911, @erikgrinaker, and @nvanbenschoten)
pkg/kv/sender.go, line 396 at r1 (raw file):
// WrappedSenderFactoryFunc is a factory function that wraps a sender in another sender. type WrappedSenderFactoryFunc func(Sender) Sender
does this type need to be in the kv
pkg? Can it be moved to kvcoord
? And actually, is the type needed at all (as opposed to inlining the func signature into ClientTestingKnobs
? The comment on the knob is awkward in the absence of the signature.
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 78 at r2 (raw file):
and subsequent requests
Are there "subsequent requests" to speak of in a rollback was sent? In case the rollback comes from the client, I think there shouldn't be. And if the rollback is the async one, I think further requests are rejected because we set h.mu.finalObservedStatus
. No?
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 88 at r1 (raw file):
defer gs.mu.Lock() sender := gs.wrapped if intercept := gs.knobs.TxnRequestInterceptorFactory; intercept != nil {
isn't it ugly to call into this sender without the lock? That makes it different from all other senders.
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.
it seems to me that adding a new interceptor for the single purpose of blocking the async EndTxn sounds complicated. Have you considered other options?
We briefly discussed this on the original issue: #65458 (comment)
it seems to me that we could have the async rollback also be pessimistic like that. So, we could have the pipeliner maintain tentative intent spans while a request is in progress, that get replaced with the real spans if a response is received.
I considered adding broader spans to the EndTxn, but chose blocking instead for two reasons:
-
Synchronous systems are much simpler and easier to reason about than async ones -- these bugs are an example of that. The
TxnCoordSender
is generally synchronous, with the sole exception of EndTxn(commit=false). If we can force EndTxn(commit=false) into the synchronous model it's a big win for simplicity and correctness in the rest of the code. -
If we send the EndTxn(commit=false) concurrently with a write, even with updated lock spans, I don't believe the write is guaranteed to be applied first -- if it isn't, it would leak intents. Maybe I'm wrong, I don't have a complete understanding of our concurrency control yet, but that sort of goes towards my point above: the synchronous model is obviously correct, the async one isn't.
Of course, the downside is that we may have to wait for some expensive (and now unnecessary) operation to complete, which could cause increased latency elsewhere (e.g. other txns waiting for this one). Context cancellation could help with this.
I think the heartbeater is the right place to have the synchronization between the async rollback and the in-flight requests. It has all the state it needs - it knows whether a request is in flight, and it controls the async rollback.
While I agree that it's borderline overkill to add an interceptor for this, and we could do this in the heartbeater, I think the big advantage of middleware stacks is that they afford clean separation of concerns. Request serialization and heartbeats do not appear to be the same concern, and even though the heartbeater sends many of the async rollbacks, it does not send all (e.g. context cancellation rollbacks). I was also thinking that we could move more of the txn finalization logic into this interceptor in the future, which would make this logic cleaner (e.g. avoid having to look inside the txnHeartbeater state). I don't know what performance overhead it has though.
If we want to avoid the extra interceptor, I personally think it'd be more natural to move it into the txnPipeliner, since request concurrency control is at least within its area of concern. However, I feel like it's probably complex enough already. I don't have a very strong opinion on this though.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @aliher1911, @andreimatei, and @nvanbenschoten)
pkg/kv/sender.go, line 396 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
does this type need to be in the
kv
pkg? Can it be moved tokvcoord
? And actually, is the type needed at all (as opposed to inlining the func signature intoClientTestingKnobs
? The comment on the knob is awkward in the absence of the signature.
I'm fine with just using a naked signature.
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 78 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
and subsequent requests
Are there "subsequent requests" to speak of in a rollback was sent? In case the rollback comes from the client, I think there shouldn't be. And if the rollback is the async one, I think further requests are rejected because we set
h.mu.finalObservedStatus
. No?
It depends on whether there are additional sources of async rollbacks. One other is context cancellation, but that's sort of from the client as well. If there are no other sources of async rollbacks, and never will be, then we could loosen some of this logic -- but I didn't want to make that assumption.
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 88 at r1 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
isn't it ugly to call into this sender without the lock? That makes it different from all other senders.
Perhaps, but that's what I need for the tests: to block a put response from being returned to the client (after it has been executed and freed up latches), and allow the TxnCoordSender to send additional requests while it's blocked. I suppose I could pass the mutex to the function so it could do the (un)locking itself, if you'd prefer -- or add/find another knob to do this in the kvserver instead.
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 @andreimatei and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 88 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Perhaps, but that's what I need for the tests: to block a put response from being returned to the client (after it has been executed and freed up latches), and allow the TxnCoordSender to send additional requests while it's blocked. I suppose I could pass the mutex to the function so it could do the (un)locking itself, if you'd prefer -- or add/find another knob to do this in the kvserver instead.
Isn't this sender outside of interceptor stack, but rather a wrapper for underlying sender? In that case calling it without lock is right. I think the name for the knob might be improved maybe? We have an interceptor stack and we have an interceptor for requests which are not the same thing.
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 @andreimatei and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 88 at r1 (raw file):
Previously, aliher1911 (Oleg) wrote…
Isn't this sender outside of interceptor stack, but rather a wrapper for underlying sender? In that case calling it without lock is right. I think the name for the knob might be improved maybe? We have an interceptor stack and we have an interceptor for requests which are not the same thing.
That's true. It'd maybe make more sense to wrap the sender before TxnCoordSender construction and pass it in.
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.
If we send the EndTxn(commit=false) concurrently with a write, even with updated lock spans, I don't believe the write is guaranteed to be applied first -- if it isn't, it would leak intents.
But that's the case anyway, isn't it? Just because you wait for a response from all the in-flight requests doesn't mean that the writes done by that request cannot be reordered with the rollback: in case of connection errors or in case the request's context is canceled, the client gets the error but the request might be on the wire. I don't believe we have any protection against this sort of reordering. The protection we do have is against reads being reordered with the rollback and missing to see intents that were cleaned up. That's what this "poisoning" does. But I don't think anything prevents writes intents from being written after the rollback. I think we could prevent this reordering if we populated the AbortSpan
on ranges where the rollback hasn't actually cleared any intents (the numKeys > 0
condition [here[(https://github.com/cockroachdb/cockroach/blob/d0ebb7f1952b3da714bc4fa801c013cfdce52300/pkg/kv/kvserver/batcheval/cmd_resolve_intent_range.go#L64). Populating the abort span has a cost, though, so I'm not sure if it's worth it. Perhaps we could do a best-effort thing where a leaseholder would have a small cache poisoning aborts.
Of course, the downside is that we may have to wait for some expensive (and now unnecessary) operation to complete, which could cause increased latency elsewhere (e.g. other txns waiting for this one). Context cancellation could help with this.
Yeah, I was also thinking that maybe we should cancel the ctx of the in-flight request. But then we make the races discussed above more likely.
While I agree that it's borderline overkill to add an interceptor for this, and we could do this in the heartbeater, I think the big advantage of middleware stacks is that they afford clean separation of concerns. Request serialization and heartbeats do not appear to be the same concern, and even though the heartbeater sends many of the async rollbacks, it does not send all (e.g. context cancellation rollbacks).
What are "context cancellation rollbacks"? We don't have any async actor watching for ctx cancellation and sending rollbacks (do we?); I think rollbacks are either sent by the client (who's not allowed to send them concurrently with any other request) or by the heartbeater.
If it's only the heartbeater that needs this request serialization, I don't think that "request serialization and heartbeats do not appear to be the same concern" is a fair statement :).
But if you still disagree, I'm fine with the new interceptor too.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 78 at r2 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
It depends on whether there are additional sources of async rollbacks. One other is context cancellation, but that's sort of from the client as well. If there are no other sources of async rollbacks, and never will be, then we could loosen some of this logic -- but I didn't want to make that assumption.
Well, I'd say that, if we ever get more sources of async rollbacks, we can modify this code to support them.
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 24 at r3 (raw file):
or client disconnects.
Do client disconnects really result in an async rollback? I'd expect that rollback to be synchronous.
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 26 at r3 (raw file):
This is necessary
Nit: s/This/Serializing rollbacks with other requests
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.
Just because you wait for a response from all the in-flight requests doesn't mean that the writes done by that request cannot be reordered with the rollback: in case of connection errors or in case the request's context is canceled, the client gets the error but the request might be on the wire.
Right, this is true -- with errors, all bets are off.
Populating the abort span has a cost, though, so I'm not sure if it's worth it.
I agree, that doesn't seem worth it here as long as we can reduce intent leakage to a reasonably low level and have them GCed within a reasonable time.
Yeah, I was also thinking that maybe we should cancel the ctx of the in-flight request. But then we make the races discussed above more likely.
Good point. This still applies for client disconnects though, we may want to try something simple like waiting until the context cancellation "should" have propagated in that case -- unless we can come up with anything better that's reasonably cheap.
What are "context cancellation rollbacks"? We don't have any async actor watching for ctx cancellation and sending rollbacks (do we?)
I was thinking of cases where the client sends a write and then immediately disconnects, in which case the SQL connection rolls back on its behalf. But of course, in this case the context cancellation will trigger deep down in the DistSender
, and the error would be propagated back up the TxnCoordSender stack before sending the EndTxn
, so this case is synchronous too -- I didn't think that all the way through.
If it's only the heartbeater that needs this request serialization, I don't think that "request serialization and heartbeats do not appear to be the same concern" is a fair statement :).
Yeah, I think you've convinced me. Let me mull on this for a bit, but if I can't come up with any counterpoints then I agree that we can special-case this in the heartbeater.
Are you still on board with serializing the heartbeater rollbacks rather than using pessimistic lock spans though? Wdyt @nvanbenschoten?
Come to think of it, this probably means that the EndTxn elision isn't actually an issue either (at least not the one in TxnCoordSender itself) -- I did in fact have to provoke that test failure by using an async rollback with a separate context.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 78 at r2 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
Well, I'd say that, if we ever get more sources of async rollbacks, we can modify this code to support them.
👍
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 24 at r3 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
or client disconnects.
Do client disconnects really result in an async rollback? I'd expect that rollback to be synchronous.
Indeed, answered this above.
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.
there are likely scenarios such as e.g. leaf txns that aren't properly handled here.
Leaf transactions cannot acquire locks, so at least on the surface, this should be ok.
If we send the EndTxn(commit=false) concurrently with a write, even with updated lock spans, I don't believe the write is guaranteed to be applied first -- if it isn't, it would leak intents. Maybe I'm wrong, I don't have a complete understanding of our concurrency control yet, but that sort of goes towards my point above: the synchronous model is obviously correct, the async one isn't.
Your understanding is correct. If intent resolution does not use the Poison flag, then it is possible for an intent to land after intent resolution has completed. However, if intent resolution does use the Poison flag, then a record of the aborted intent will be stored in each range that clears intents to ensure that zombie transactions read-their-writes.
EDIT: I see Andrei already explained this.
I think the heartbeater is the right place to have the synchronization between the async rollback and the in-flight requests. It has all the state it needs - it knows whether a request is in flight, and it controls the async rollback.
I tend to agree with this. I also found the additional interceptor to be unfortunate.
One idea to keep the logic all within the heartbeater without the need to be quite as explicit about synchronization and waiting is to keep track of the number of outstanding requests that have passed through the heartbeater - I guess this should be 1 or 0. When the heartbeat loop notices that a transaction is aborted, it checks this count. If the count is 0, it calls into abortTxnAsyncLocked
. Otherwise, it simply leaves a note (needAsyncTxnAbort
?) for the in-flight response and exits. When a response passes through the heartbeater, it decrements the count. If the count is 0, it checks the needAsyncTxnAbort
flag and, if set, kicks off the async abort. This strategy effectively drains actors at or below the heartbeater and has the last man standing kick off the async abort.
Are you still on board with serializing the heartbeater rollbacks rather than using pessimistic lock spans though? Wdyt @nvanbenschoten?
At this point, I still think we should explore serializing in the heartbeater as a preferred approach.
Reviewed 4 of 4 files at r1, 4 of 5 files at r2.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 71 at r1 (raw file):
if !gs.allowConcurrentRequests { asyncRequest := ba.IsSingleAbortTxnRequest() || ba.IsSingleHeartbeatTxnRequest() if !asyncRequest {
👍 nice!
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 88 at r1 (raw file):
It'd maybe make more sense to wrap the sender before TxnCoordSender construction and pass it in.
This seems to make more sense to me. If we want to mock out or instrument the sender that is passed to the TxnCoordSender, give it a different sender.
Also, I wonder if there are other testing knobs that you could use to accomplish the same task. There are plenty of knobs that you can use to block responses on the server. Did you see TestingResponseFilter
?
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.
If we send the EndTxn(commit=false) concurrently with a write, even with updated lock spans, I don't believe the write is guaranteed to be applied first -- if it isn't, it would leak intents. Maybe I'm wrong, I don't have a complete understanding of our concurrency control yet, but that sort of goes towards my point above: the synchronous model is obviously correct, the async one isn't.
Your understanding is correct. If intent resolution does not use the Poison flag, then it is possible for an intent to land after intent resolution has completed. However, if intent resolution does use the Poison flag, then a record of the aborted intent will be stored in each range that clears intents to ensure that zombie transactions read-their-writes.
EDIT: I see Andrei already explained this.
For the sake of clarity - we're all in agreement that, Poison
or no poison, leaking intents through a write batch being reordered with a rollback (async rollback or regular rollback) is possible when a network error or client ctx cancelation causes the client to not actually wait for the respective batch RPC to complete before sending the rollback, right?
Come to think of it, this probably means that the EndTxn elision isn't actually an issue either (at least not the one in TxnCoordSender itself) -- I did in fact have to provoke that test failure by using an async rollback with a separate context.
Sorry, what elision are we talking about? Feel free to ignore if things are clear :)
One idea to keep the logic all within the heartbeater without the need to be quite as explicit about synchronization and waiting is to keep track of the number of outstanding requests that have passed through the heartbeater - I guess this should be 1 or 0. When the heartbeat loop notices that a transaction is aborted, it checks this count. If the count is 0, it calls into abortTxnAsyncLocked. Otherwise, it simply leaves a note (needAsyncTxnAbort?) for the in-flight response and exits. When a response passes through the heartbeater, it decrements the count. If the count is 0, it checks the needAsyncTxnAbort flag and, if set, kicks off the async abort. This strategy effectively drains actors at or below the heartbeater and has the last man standing kick off the async abort.
This is also what I had in mind.
Are you still on board with serializing the heartbeater rollbacks rather than using pessimistic lock spans though? Wdyt @nvanbenschoten?
At this point, I still think we should explore serializing in the heartbeater as a preferred approach.
I'm on board, yeah.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, 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.
I tend to agree with this. I also found the additional interceptor to be unfortunate.
I agree. I thought we would have to handle arbitrary async rollback requests, but I was wrong, so we can deal with this trivially in txnHeartbeater
. Updating the PR shortly.
Come to think of it, this probably means that the EndTxn elision isn't actually an issue either (at least not the one in TxnCoordSender itself)
Sorry, what elision are we talking about? Feel free to ignore if things are clear :)
This one isn't racy, because client rollbacks are synchronous:
This was racy, but will be fixed by the txnHeartbeater serialization:
cockroach/pkg/kv/kvclient/kvcoord/txn_interceptor_committer.go
Lines 130 to 135 in ee23325
// Determine whether we can elide the EndTxn entirely. We can do so if the | |
// transaction is read-only, which we determine based on whether the EndTxn | |
// request contains any writes. | |
if len(et.LockSpans) == 0 && len(et.InFlightWrites) == 0 { | |
return tc.sendLockedWithElidedEndTxn(ctx, ba, et) | |
} |
we're all in agreement that,
Poison
or no poison, leaking intents through a write batch being reordered with a rollback (async rollback or regular rollback) is possible when a network error or client ctx cancelation causes the client to not actually wait for the respective batch RPC to complete before sending the rollback, right?
I'll defer to @nvanbenschoten here, have to read up on abort spans. But I'd like to discuss if and how we should handle this -- seems to me like it's edge casey enough that we can accept leaking these intents, but if it's as simple as setting Poison: true
and eating a small write cost then maybe it's worth considering.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 88 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
It'd maybe make more sense to wrap the sender before TxnCoordSender construction and pass it in.
This seems to make more sense to me. If we want to mock out or instrument the sender that is passed to the TxnCoordSender, give it a different sender.
Also, I wonder if there are other testing knobs that you could use to accomplish the same task. There are plenty of knobs that you can use to block responses on the server. Did you see
TestingResponseFilter
?
Ah, thanks! Don't know how I missed those.
98b53dd
to
c14ea2d
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.
Rewrote the PR, PTAL.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
504c945
to
239d6df
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.
Tests caught a concurrency violation, as the TxnCoordSender allows EndTxn(commit=false) from the client even with finalObservedStatus
set:
cockroach/pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Lines 648 to 654 in cfb4b95
if ba != nil && ba.IsSingleAbortTxnRequest() { | |
// As a special case, we allow rollbacks to be sent at any time. Any | |
// rollback attempt moves the TxnCoordSender state to txnFinalized, but higher | |
// layers are free to retry rollbacks if they want (and they do, for | |
// example, when the context was canceled while txn.Rollback() was running). | |
return nil | |
} |
I guess our options are:
- Serialize the request with the previous EndTxn(commit=false) request in the txnHeartbeater.
- Drop the txnLockGatekeeper concurrency assertion for EndTxn(commit=false) requests.
- Return a fake response when
finalObservedStatus=ABORTED
.
1 seems safest, 2 easiest (and safer to backport). 3 seems fine, but I'm not sure if I understand all of the implications.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
31a5044
to
e5993e4
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.
I got that, but it was at odds with what Nathan suggested, and I didn't have an informed opinion. Looked into it now, and you're right (as you pointed out): it won't prevent intent leaks when the write gets delayed after a poisoned rollback and the rollback doesn't clear any previously written intents.
My sense is that we should write the abort span regardless of removed intents, since its intended purpose is to prevent exactly these sorts of reorderings, and update the context cancellation rollback to poison as well. It does add some cost, but it appears small (at most one additional write per range), and there's no client around to care about latency anymore. EndTxn elision also eliminates this cost for read-only txns.
"and update the context cancellation rollback to poison as well" - why just the ctx cancelation rollback and not all rollbacks (or at least all rollbacks done after any errors were encountered by any request)? Is the ctx cancelation rollback any more likely to be reordered with some phantom write than any other rollback? I guess maybe you can argue that it is because, if a pgwire connection's ctx was canceled, then it's likely that an RPC was interrupted without waiting for its completion and so the rollback we send might race with that RPC. In other words, canceling a connection's ctx while a query is running can lead to the rollback being sent quickly after some other RPC.
In any case, I dunno how likely these reorderings really are; for a reordering to occur, I think you'd need a new network connection to be established before the other RPC acquires its latches. I'm also not sure it'd be a good idea to poison more. The latency cost may be small, but the data size cost might be significant. If you have a workload where your transactions are constantly timing out, depending on what policy we'd end up with for poisoning, we might write a lot of abort records.
I now regret bringing these reorderings up; I forgot what was the point I originally wanted to make with them :)
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 216 at r5 (raw file):
h.cancelHeartbeatLoopLocked() // If an abortTxnAsyncLocked() rollback is in flight, we'll wait
My preference would be to actually prefer this rollback over the abortTxnAsyncLocked
one, because this one doesn't poison. Besides the poisoning aspect, prefering the client's rollback matches better with how I think about the async rollback - namely, I see the async rollback as opportunistic, meant only to clean up a doomed transaction quicker than the client would otherwise (because it might take the client a long time to send its rollback - in case of DistSQL flows, all the remote flows need to finish and drain and whatever before the control returns to the gateway to rollback). But if the client has sent a rollback, the point is moot; there's no longer a reason to send the "opportunistic" async one.
Feel free to ignore or disagree. There's been a bunch of back and forth and I don't care too much.
To be honest, I could also be convinced to drop the async rollback completely since it's giving us headaches. (but we'd keep the rejectClientLocked
mechanism so that the client can find out that the transaction is aborted before trying (and failing) to commit it.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 217 at r5 (raw file):
// If an abortTxnAsyncLocked() rollback is in flight, we'll wait // for the result here to avoid violating the txnLockGatekeeper
If you keep the code like this, please update this comment to spell out that this rollback requests doesn't actually run, and instead gets the result of the other one.
You might also want to expand on "avoid violating the txnLockGatekeeper" - I don't think people will understand what that means (I don't think I do either). Or just strike the comment; and just say that we don't need two rollbacks.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 220 at r5 (raw file):
// concurrency assertion. if resultC := h.mu.abortTxnAsyncResultC; resultC != nil { h.mu.Unlock()
nit: This unusual unlock deserves a comment. Maybe here say that this is normally the gatekeeper's job.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 234 at r5 (raw file):
// Record the in-flight request and its response. h.mu.ifReqs++ defer func() {
nit: instead of the defer
, I think it'd read better if you just move this code a line down, after the SendLocked
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 459 at r5 (raw file):
} // If another abortTxnAsyncLocked attempt is already in flight, bail out.
nit: I'd move this stanza up, as the first thing in the method. I think it's nice to immediately see that a method is idempotent.
Also, should we check abortTxnAsyncPending
in addition to abortTxnAsyncResultC
?
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 463 at r5 (raw file):
// independently of the heartbeat loop. if h.mu.abortTxnAsyncResultC != nil { log.VEventf(ctx, 1, "Heartbeat detected aborted txn, but found concurrent abort attempt.")
This comment seems to make the assumption that, when this method is called twice, the 2nd call comes from the heartbeat loop. But is that necessarily the case? Consider just saying "txn already rolling back" or so.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 467 at r5 (raw file):
} log.VEventf(ctx, 1, "Heartbeat detected aborted txn. Cleaning up.")
nit: Since you're making this a VEventf
instead of an Eventf
(and so it might make it to the logs), please include txn.ID.Short()
in the message. Showing up in a trace without more context is fine (cause the trace has the context), but if you ever see it in a log, you're gonna want to know what txn it's talking about.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 513 at r5 (raw file):
); err != nil { log.Warningf(ctx, "%v", err) h.mu.abortTxnAsyncResultC = nil // task wasn't started after all
nit: consider only assigning h.mu.abortTxnAsyncResultC
once, if there's no error from RunAsyncTask
.
e5993e4
to
7490233
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.
TFTRs, appreciate you taking the time to be thorough! Updated the PR, I'm going to let this stew overnight and look over it again with fresh eyes tomorrow.
why just the ctx cancelation rollback and not all rollbacks (or at least all rollbacks done after any errors were encountered by any request)?
I think it's more likely for context cancelation (and network errors), since typical error responses tend to originate at the far bottom of the RPC call stack, while contexts can get cancelled by selects anywhere along the path (e.g. at RPC or Raft boundaries). But yes, to be comprehensive we'd have to do it for all errors.
In any case, I dunno how likely these reorderings really are; for a reordering to occur, I think you'd need a new network connection to be established before the other RPC acquires its latches. I'm also not sure it'd be a good idea to poison more.
I agree, they're likely to be very rare. Let's drop it for now, and we can revisit if we're still seeing significant intent leaks once all the other fixes are in.
Reviewable status:
complete! 1 of 0 LGTMs obtained (waiting on @andreimatei, @erikgrinaker, and @nvanbenschoten)
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 78 at r2 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
👍
Done.
pkg/kv/kvclient/kvcoord/txn_interceptor_finalize_barrier.go, line 24 at r3 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Indeed, answered this above.
Done.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 216 at r5 (raw file):
My preference would be to actually prefer this rollback over the abortTxnAsyncLocked one, because this one doesn't poison. Besides the poisoning aspect, prefering the client's rollback matches better with how I think about the async rollback - namely, I see the async rollback as opportunistic, meant only to clean up a doomed transaction quicker than the client would otherwise
Makes sense, moved the synchronization into the async abort goroutine so that whoever wins the mutex race goes first.
To be honest, I could also be convinced to drop the async rollback completely since it's giving us headaches.
At this point, I'd rather not accidentally introduce new intent leaks. :) But it's worth revisiting, it does add quite a bit of complexity into an already complex code path. I'd like to restructure intent resolution as a queue sometime, maybe we could clean this up then.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 217 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
If you keep the code like this, please update this comment to spell out that this rollback requests doesn't actually run, and instead gets the result of the other one.
You might also want to expand on "avoid violating the txnLockGatekeeper" - I don't think people will understand what that means (I don't think I do either). Or just strike the comment; and just say that we don't need two rollbacks.
Done.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 220 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: This unusual unlock deserves a comment. Maybe here say that this is normally the gatekeeper's job.
Good call, done.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 234 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: instead of the
defer
, I think it'd read better if you just move this code a line down, after theSendLocked
Thanks, that does reads better.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 459 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: I'd move this stanza up, as the first thing in the method. I think it's nice to immediately see that a method is idempotent.
Also, should we check
abortTxnAsyncPending
in addition toabortTxnAsyncResultC
?
No longer applies, moved into goroutine to give client rollbacks a chance to go first.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 463 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
This comment seems to make the assumption that, when this method is called twice, the 2nd call comes from the heartbeat loop. But is that necessarily the case? Consider just saying "txn already rolling back" or so.
You're right, these log messages are a bit outdated. I moved the ones mentioning heartbeat failures to the relevant call sites, and reworded and expanded some messages.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 467 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: Since you're making this a
VEventf
instead of anEventf
(and so it might make it to the logs), please includetxn.ID.Short()
in the message. Showing up in a trace without more context is fine (cause the trace has the context), but if you ever see it in a log, you're gonna want to know what txn it's talking about.
Good call, done.
pkg/kv/kvclient/kvcoord/txn_interceptor_heartbeater.go, line 513 at r5 (raw file):
Previously, andreimatei (Andrei Matei) wrote…
nit: consider only assigning
h.mu.abortTxnAsyncResultC
once, if there's no error fromRunAsyncTask
.
No longer applies.
pkg/kv/kvclient/kvcoord/txn_lock_gatekeeper.go, line 88 at r1 (raw file):
Previously, erikgrinaker (Erik Grinaker) wrote…
Ah, thanks! Don't know how I missed those.
Done.
7490233
to
909c46c
Compare
bors r=andreimatei |
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 9 files at r4, 1 of 2 files at r5, 2 of 2 files at r6.
Reviewable status:complete! 1 of 0 LGTMs obtained (and 1 stale) (waiting on @andreimatei and @nvanbenschoten)
@erikgrinaker @andreimatei what's your take on these backports? This is a great change, but it's also fairly large and has a lot of moving parts. We always have to weigh the risk vs. the reward with backports, and now that #64131 has landed and has been backported, that tradeoff isn't as clear cut. Given the complexity of this change, my take is that we should let it bake on master for 3 or 4 weeks before trying to get this into patch releases. |
I feel the same way. I opened draft backport PRs just to have them ready, but given how critical this code path is, let's not rush them into the next patch releases -- they can wait for the next round. |
Build failed (retrying...): |
bors r- Rare test failure, txn record cleanup failure. Thought it had been resolved, but apparently not. |
Canceled. |
909c46c
to
f8f561c
Compare
`TxnCoordSender` generally operates synchronously (i.e. the client waits for the previous response before sending the next request). However, the `txnHeartbeater` sends asynchronous `EndTxn(commit=false)` rollbacks when it discovers an aborted transaction record. Unfortunately, some code assumes synchrony, which caused race conditions with txn rollbacks. In particular, the `txnPipeliner` attaches lock spans and in-flight writes to the `EndTxn` request for e.g. intent cleanup, but it only records this information when it receives responses. Thus, if an `EndTxn(commit=false)` is sent concurrently with a write request, the lock spans and in-flight writes of that write request will not get attached to the `EndTxn` request and the intents will not get cleaned up. This patch makes the `txnHeartbeater` wait for any in-flight requests to complete before sending asynchronous rollbacks, and collapses incoming client rollbacks with in-flight async rollbacks. Release note (bug fix): Fixed a race condition where transaction cleanup would fail to take into account ongoing writes and clean up their intents.
f8f561c
to
f2c6b45
Compare
bors r=andreimatei,nvanbenschoten |
Build succeeded: |
TxnCoordSender
generally operates synchronously (i.e. the client waitsfor the previous response before sending the next request). However, the
txnHeartbeater
sends asynchronousEndTxn(commit=false)
rollbackswhen it discovers an aborted transaction record. Unfortunately, some code
assumes synchrony, which caused race conditions with txn rollbacks.
In particular, the
txnPipeliner
attaches lock spans and in-flightwrites to the
EndTxn
request for e.g. intent cleanup, but it onlyrecords this information when it receives responses. Thus, if an
EndTxn(commit=false)
is sent concurrently with a write request, thelock spans and in-flight writes of that write request will not get
attached to the
EndTxn
request and the intents will not get cleanedup.
This patch makes the
txnHeartbeater
wait for any in-flight requests tocomplete before sending asynchronous rollbacks, and collapses incoming
client rollbacks with in-flight async rollbacks.
Resolves #65458.
Resolves #65587.
Resolves #65447.
Release note (bug fix): Fixed a race condition where transaction cleanup
would fail to take into account ongoing writes and clean up their
intents.
/cc @cockroachdb/kv