-
Notifications
You must be signed in to change notification settings - Fork 3.8k
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
sql, roachpb: correctly marshall retryable errors through DistSQL #15187
sql, roachpb: correctly marshall retryable errors through DistSQL #15187
Conversation
sending for review while I figure out how to test this sucker |
Reviewed 1 of 2 files at r1, 11 of 11 files at r2. pkg/kv/dist_sender_server_test.go, line 1152 at r2 (raw file):
Why did you remove this? pkg/roachpb/errors.proto, line 98 at r2 (raw file):
s/righ/right/ Comments from Reviewable |
distsql part LGTM Review status: all files reviewed at latest revision, 2 unresolved discussions, some commit checks failed. Comments from Reviewable |
what I've done here is wholly insufficient; I'm ignoring the other updates performed by the Review status: all files reviewed at latest revision, 2 unresolved discussions, some commit checks failed. pkg/roachpb/errors.proto, line 98 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. Comments from Reviewable |
4720e35
to
7562c4d
Compare
Pushed another commit that makes distSQL compensate for the TxnCoordSender. PTAL r3 is #15305 Review status: 0 of 22 files reviewed at latest revision, 2 unresolved discussions. pkg/kv/dist_sender_server_test.go, line 1152 at r2 (raw file): Previously, bdarnell (Ben Darnell) wrote…
what I had uploaded was incomplete here. The type of the error changes and code like this is no longer supposed to peek inside the error. But the test still tests this Comments from Reviewable |
A test with a DistSQL restart is still forthcoming. Review status: 0 of 22 files reviewed at latest revision, 2 unresolved discussions, some commit checks pending. Comments from Reviewable |
This makes me nervous. I've always been skeptical about the push to combine client.Txn and kv.TxnCoordSender because client.Txn was fairly thin and didn't have that much overlap with TxnCoordSender. This change introduces significant overlap between them (so I guess it demonstrates the necessity of the merge), and now distsql writes updates to client.Txn that only very indirectly make it in to TxnCoordSender (TxnCoordSender won't update its txn proto until the next command returns, I think). I think I'd be more comfortable if we could make Reviewed 4 of 12 files at r4, 18 of 18 files at r5. pkg/internal/client/txn.go, line 78 at r5 (raw file):
s/Aborted/Switched/ pkg/internal/client/txn.go, line 936 at r5 (raw file):
It seems like this needs to be resolved before this can be merged. I think we need to make sure that the result of PrepapreTransactionForRetry moves the txn forward in time before assigning it to txn.mu.Proto: it either changes the ID or increases the epoch. pkg/internal/client/txn_test.go, line 588 at r5 (raw file):
Why did this need to be added here? Have we lost something on the non-test path? pkg/roachpb/errors.proto, line 350 at r5 (raw file):
Could we move towards this design today by having this error contain a Comments from Reviewable |
is depending on this PR to provide a mechanism to freshly retry a transaction within txn.Exec() when the closure run by Exec() returns a retryable error from the sql layer (getTableLease() can push the transaction forward). We need code that will create a new txn with a new id, and which will abort the previous incarnation of the transaction. Ideally we would like the intents to also be GC-ed but that's not a hard requirement given that we don't expect this retry to happen very often (only on txns running concurrent with a schema change potentially) |
7562c4d
to
5d89c15
Compare
Ben, I've moved the restarting logic back into TxnCoordSender. PTAL. Review status: 7 of 25 files reviewed at latest revision, 5 unresolved discussions, some commit checks failed. Comments from Reviewable |
This is looking better. My main concern is that RetryableTxnError and InternalRetryableTxnError are confusingly named. Do we need both, and if so can we better articulate the difference between them? (HandledRetryableTxnError makes more sense, although see my comments below). Reviewed 18 of 18 files at r6. pkg/internal/client/sender.go, line 31 at r6 (raw file):
Don't forget this. pkg/internal/client/txn.go, line 941 at r6 (raw file):
What's the plan here? pkg/kv/txn_coord_sender.go, line 516 at r6 (raw file):
I think it should be pkg/kv/txn_coord_sender.go, line 519 at r6 (raw file):
This process seems awfully convoluted, but I guess it's OK and nicely localized for now. pkg/kv/txn_coord_sender.go, line 841 at r6 (raw file):
It makes me really nervous that this is done in terms of GoError instead of ErrorDetail. pkg/roachpb/data.go, line 973 at r6 (raw file):
Which param? pkg/roachpb/errors.go, line 169 at r6 (raw file):
Why are HandledRetryableErrors turned into RetryableTxnError? HandledRetryableError implements the This use of RetryableTxnError seems very different from the use case documented in the comment for NewRetryableTxnError, which seems like it's just for the sql pkg/roachpb/errors.go, line 444 at r6 (raw file):
TransactionRestart_BACKOFF is probably a safer default. Or maybe HandledRetryableError should not implement canRestartTransaction, because the restart has already been "handled" and upper layers don't need to handle it again. pkg/roachpb/errors.proto, line 350 at r5 (raw file): Previously, bdarnell (Ben Darnell) wrote…
So it's not feasible to use the Transaction here for everything instead of new dedicated fields? pkg/roachpb/errors.proto, line 314 at r6 (raw file):
This "Internal" error seems to be used in a lot of places. Maybe we should swap the names of InternalRetryableTxnError and RetryableTxnError? (which at least within the scope of this diff is not used in nearly as many places) pkg/roachpb/errors.proto, line 369 at r6 (raw file):
This field doesn't appear to ever be used. pkg/sql/distsqlrun/base.go, line 630 at r6 (raw file):
Yeah, that sounds right (assuming "global" means "on the original gateway" and "local" means "local to the distsql worker node"). There should only be one TxnCoordSender involved. Comments from Reviewable |
5d89c15
to
62ed249
Compare
I got rid of RetryableTxnError. It was technically useful in separating some layers, but I agree that it was confusing. See if you like it better please. I'm working on the DistSQL code path now... Seeing how to bypass the Review status: 13 of 32 files reviewed at latest revision, 15 unresolved discussions, some commit checks failed. pkg/internal/client/txn.go, line 936 at r5 (raw file): Previously, bdarnell (Ben Darnell) wrote…
PrepareTxnForRetry does do what you say. And we now check that the epoch is not stale before updating the state. pkg/internal/client/txn_test.go, line 588 at r5 (raw file): Previously, bdarnell (Ben Darnell) wrote…
this was added because now a call to GoError() does what TxnCoordSender used to do and checks for that observed timestamp. The tests in this file without the TxnCS are pretty hacky, since the client.Txn does rely on the TxnCS to do stuff on the error path. insert rant about cient.Txn and TCS being one thing. pkg/kv/txn_coord_sender.go, line 516 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
pkg/kv/txn_coord_sender.go, line 519 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
the real problem, I think, is that the TxnCS and the client.Txn are forced to communicate through the narrow Sender interface. pkg/kv/txn_coord_sender.go, line 841 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
the idea is that this code (or at least the
and the actual update of TxnCoordSender's map will be shared with the DistSQL backdoor path. And DistSQL never has access to a pErr, only to a Why does this make you so nervous? pkg/roachpb/data.go, line 973 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
yes, clarified. pkg/roachpb/errors.go, line 169 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Note that the comment about But I've gotten rid of pkg/roachpb/errors.go, line 444 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Well the restart has been "handled" in the sense that a But I've removed the pkg/roachpb/errors.proto, line 350 at r5 (raw file): Previously, bdarnell (Ben Darnell) wrote…
My TODO was about the communication between storage and KV (through pErr.Transaction), not about this error that's used above storage (so that As to whether we could use the pkg/roachpb/errors.proto, line 314 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
got rid of pkg/roachpb/errors.proto, line 369 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
hmmm yes good catch. I've put this here because, before this change, the That line seems nonsensical to me, because a) So, I've now removed this field. Comments from Reviewable |
Reviewed 19 of 19 files at r7. pkg/kv/txn_coord_sender.go, line 516 at r6 (raw file): Previously, andreimatei (Andrei Matei) wrote…
Ah, I see. I think the right thing to do is to remove the UserPriority argument from PrepareTransactionForRetry, always use the minimum priority, then when the pkg/kv/txn_coord_sender.go, line 519 at r6 (raw file): Previously, andreimatei (Andrei Matei) wrote…
Yeah, I've been questioning the Sender interface for a long time. pkg/kv/txn_coord_sender.go, line 841 at r6 (raw file): Previously, andreimatei (Andrei Matei) wrote…
It makes me nervous because it's contrary to the design for roachpb.Error and for error handling in the kv and storage packages. We never used the non-proto RetryableTransactionError in the kv package before. As I've said before, I think it's unfortunate that distsql has diverged from this model, but given that we should still try to keep the error handling model within the kv package the same as it was. To that end, my suggestion is to undo the changes to pkg/roachpb/errors.go, line 169 at r6 (raw file): Previously, andreimatei (Andrei Matei) wrote…
I don't think this needs to be a special case any more (now that HandledRetryableTxnError no longer sets e.TransactionRestart); it can just be handled by the pkg/roachpb/errors.go, line 71 at r7 (raw file):
What's the plan here? pkg/roachpb/errors.proto, line 350 at r5 (raw file): Previously, andreimatei (Andrei Matei) wrote…
It would be a partially-initialized field today, but it might become more fully-initialized in the future (i.e. we could have TxnAbortedError generate the ID for the next attempt and return a fully-initialized proto in addition to the ID of the aborted txn). Using the Txn proto would make it easier to start propagating more information across aborts (like observed timestamps). But I don't feel strongly about this and sticking with separate fields for now is fine too. pkg/roachpb/errors.proto, line 314 at r6 (raw file): Previously, andreimatei (Andrei Matei) wrote…
But what is the scope that it is internal to? It's higher-level than roachpb.Error and is used in more layers instead of being more or less contained within kv and storage. pkg/roachpb/errors.proto, line 369 at r6 (raw file): Previously, andreimatei (Andrei Matei) wrote…
The linked line was redundant before, but is it now? The I think the fact that client.Txn was ignoring the timestamp in the error's txn was a bug, but a small one (the error's txn has a timestamp that we won't get anywhere else, but it's unlikely to be greater than the one we'd get from the hlc on the next call to Send). Comments from Reviewable |
62ed249
to
a2e40cc
Compare
Review status: 15 of 36 files reviewed at latest revision, 12 unresolved discussions, some commit checks failed. pkg/internal/client/sender.go, line 31 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/internal/client/txn.go, line 941 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
see now pls pkg/kv/txn_coord_sender.go, line 516 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
I don't know if we should involve the TxnCoordSender in the restart like this. In any case, the UserPriority is not used in the abort case, which is this case. pkg/kv/txn_coord_sender.go, line 841 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/roachpb/errors.go, line 169 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
Done. pkg/roachpb/errors.go, line 71 at r7 (raw file): Previously, bdarnell (Ben Darnell) wrote…
see now pls pkg/roachpb/errors.proto, line 314 at r6 (raw file): Previously, bdarnell (Ben Darnell) wrote…
renamed to... drums... DistSQLRetryableTxnError. Cause DistSQL is the only one that's supposed to ever see it. Curious if you'll dig. pkg/roachpb/errors.proto, line 369 at r6 (raw file):
The line Comments from Reviewable |
(mainly for the kv/client/roachpb parts - you may want someone else to look at the distsql parts) Thanks for being patient and working through this! Reviewed 21 of 21 files at r8. pkg/roachpb/errors.proto, line 314 at r6 (raw file): Previously, andreimatei (Andrei Matei) wrote…
👍 Comments from Reviewable |
a2e40cc
to
dcf85cd
Compare
Thanks Ben! I've added a test on the DistSQL side. Will continue thinking of other testing ideas. Review status: 29 of 38 files reviewed at latest revision, 2 unresolved discussions, some commit checks failed. Comments from Reviewable |
Retryable errors are recognized by both the TxnCoordSender and by the sql.Executor, which react to them. When these errors flow through DistSQL, the TxnCoordSender is bypassed and the Executor, before this patch, was seeing an opaque error it wouldn't recognize. This patch adds proper marshalling for these structured errors in the form of a new proto and provides a way for the distSQLReceiver to update the client.Txn's proto on errors.
26f0e1b
to
78648fb
Compare
…ble errors - TxnCoordSender becomes the only place that deals with restarting a Transaction. It takes the information from the error described above and sometimes calls Transaction.Restart(). The role of the client.Txn is reduced, and restarting-like things are consolidated in the TxnCoordSender. In the non-DistSQL world, the flow of a retryable txn becomes: - storage returns a retryable error such as TransactionRetryError or TransactionRestartedError. These are ErrorDetails in a pErr, and pErr.TransactionRestart != NONE identifies them. - TxnCoordSender.Send() recognizes these errors, takes the error's Transaction in consideration too, performs Transaction.Restart() and prepares a restarted Transaction proto. - TxnCoordSender then puts the restarted Transaction in a HandledRetryableError. This error is an ErrorDetail, and it's used for the TxnCoordSender to communicate with the client.Txn.Send() through the Sender interface. The restarted txn is also copied to its map of ongoing transactions. This way, the map is kept in sync with the Transaction that the client.Txn will use. - client.Txn now only deals with HandledRetryableError's, in terms of retryable errors. How it handles them is straight-forward: it copies the Transaction from them to its copy of the Proto (this is the proto that will be used for all future requests. On the DistSQL path, the original pErr was transformed by a pErr.GoError(), but that GoError() return value retains enough information about the cause such that the restart can be properly done on the gateway. This error is marshalled to the gateway, which uses a backdoor into client.Txn to perform the restart and update the client.Txn state.
78648fb
to
659d611
Compare
Awesome! Great work!! |
Great stuff Andrei!
…On Fri, Apr 28, 2017, 11:54 AM RaduBerinde ***@***.***> wrote:
Awesome! Great work!!
—
You are receiving this because you commented.
Reply to this email directly, view it on GitHub
<#15187 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/ALOpBE-lVoOOqYDFmCoUt0sm0G1IsCVbks5r0gujgaJpZM4NCReN>
.
|
cc @nvanbenschoten