Skip to content
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

Merged
merged 2 commits into from
Apr 28, 2017

Conversation

andreimatei
Copy link
Contributor

@andreimatei andreimatei requested a review from RaduBerinde April 19, 2017 21:35
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@andreimatei
Copy link
Contributor Author

sending for review while I figure out how to test this sucker

@bdarnell
Copy link
Contributor

Reviewed 1 of 2 files at r1, 11 of 11 files at r2.
Review status: all files reviewed at latest revision, 2 unresolved discussions, some commit checks failed.


pkg/kv/dist_sender_server_test.go, line 1152 at r2 (raw file):

				if _, ok := retErr.Cause.(*roachpb.ReadWithinUncertaintyIntervalError); ok {
					if !retErr.Transaction.Writing {
						t.Errorf("unexpected non-writing txn on error")

Why did you remove this?


pkg/roachpb/errors.proto, line 98 at r2 (raw file):

// In contrast with other errors, the Transaction that the client gets in the
// pErr carrying this ErrorDetail is not supposed to be used by the client,
// besides checking that the transaction's ID is righ.

s/righ/right/


Comments from Reviewable

@RaduBerinde
Copy link
Member

distsql part LGTM


Review status: all files reviewed at latest revision, 2 unresolved discussions, some commit checks failed.


Comments from Reviewable

@andreimatei
Copy link
Contributor Author

what I've done here is wholly insufficient; I'm ignoring the other updates performed by the TxnCoordSender to the Transaction, besides calling Transaction.UpdateStateOnErr. Stay tuned.


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…

s/righ/right/

Done.


Comments from Reviewable

@andreimatei
Copy link
Contributor Author

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…

Why did you remove this?

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 Writing bit where it matters.


Comments from Reviewable

@andreimatei
Copy link
Contributor Author

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

@bdarnell
Copy link
Contributor

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 client.Txn reach into DB.Sender to call TxnCoordSender.updateState instead of introducing more duplicative logic in client.Txn


Reviewed 4 of 12 files at r4, 18 of 18 files at r5.
Review status: all files reviewed at latest revision, 5 unresolved discussions, some commit checks failed.


pkg/internal/client/txn.go, line 78 at r5 (raw file):

		commandCount int

		// onKVTxnAborted, if set, is called when a KV-level roachpb.Transaction has

s/Aborted/Switched/


pkg/internal/client/txn.go, line 936 at r5 (raw file):

// higher priority.
//
// TODO(andrei,nvanbenschoten): document whether calling this repeatedly within

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):

						if count == 1 {
							var pErr *roachpb.Error
							if _, ok := test.err.(*roachpb.ReadWithinUncertaintyIntervalError); ok {

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):

  // will also not be set.
  //
  // NOTE(andrei): I think a better design would be for the server to always be

Could we move towards this design today by having this error contain a Transaction and a bool aborted instead of either a Transaction or a set of txn_id, retry_priority, retry_timestamp? If aborted is true, only those fields would be filled in (and PrepareTransactionForRetry would know this)


Comments from Reviewable

@vivekmenezes
Copy link
Contributor

#15254

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)

@andreimatei andreimatei force-pushed the distsql-retryable-err branch from 7562c4d to 5d89c15 Compare April 27, 2017 03:26
@andreimatei
Copy link
Contributor Author

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

@bdarnell
Copy link
Contributor

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.
Review status: all files reviewed at latest revision, 15 unresolved discussions, some commit checks failed.


pkg/internal/client/sender.go, line 31 at r6 (raw file):

}

// !!! comment

Don't forget this.


pkg/internal/client/txn.go, line 941 at r6 (raw file):

// old epochs and/or the current epoch?
func (txn *Txn) UpdateStateOnDetachedErr(ctx context.Context, err error, pri roachpb.UserPriority) {
	// !!!

What's the plan here?


pkg/kv/txn_coord_sender.go, line 516 at r6 (raw file):

		abortedErr := roachpb.NewErrorWithTxn(roachpb.NewTransactionAbortedError(), &txn)
		retryErr := abortedErr.GoError().(*roachpb.InternalRetryableTxnError)
		// TODO(andrei): figure out a UserPriority to use here.

I think it should be txnMeta.txn.Priority. That will be updated when we learn about the current transaction from the server.


pkg/kv/txn_coord_sender.go, line 519 at r6 (raw file):

		newTxn := roachpb.PrepareTransactionForRetry(
			retryErr, roachpb.NormalUserPriority, txn.Isolation, txn.Name)
		return roachpb.NewError(roachpb.NewHandledRetryableError(

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):

		var retryErr *roachpb.InternalRetryableTxnError
		if pErr.TransactionRestart != roachpb.TransactionRestart_NONE {
			goErr := pErr.GoError()

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):

// isolation and name help initialize this new transaction.
//
// TODO(andrei): This param is confusing: it's passed as the priority of the

Which param? pri?


pkg/roachpb/errors.go, line 169 at r6 (raw file):

	if e.TransactionRestart != TransactionRestart_NONE {
		if tErr, ok := e.GetDetail().(*HandledRetryableError); ok {
			return &RetryableTxnError{Msg: tErr.Msg, TxnID: tErr.TxnID}

Why are HandledRetryableErrors turned into RetryableTxnError? HandledRetryableError implements the error interface so it looks like it should be fine to return the detail directly here.

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 force_retry function and other testing.


pkg/roachpb/errors.go, line 444 at r6 (raw file):

func (*HandledRetryableError) canRestartTransaction() TransactionRestart {
	// TODO(andrei): make this dependent on the type of the original error
	return TransactionRestart_IMMEDIATE

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…

Could we move towards this design today by having this error contain a Transaction and a bool aborted instead of either a Transaction or a set of txn_id, retry_priority, retry_timestamp? If aborted is true, only those fields would be filled in (and PrepareTransactionForRetry would know this)

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):

}

// InternalRetryableTxnError tells the recipient that the transaction it

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):

  optional int32 retry_priority = 4;
  // If set, a retry should use this timestamp.
  optional util.hlc.Timestamp retry_timestamp = 5;

This field doesn't appear to ever be used.


pkg/sql/distsqlrun/base.go, line 630 at r6 (raw file):

		// local TxnCoordSender which always converts these to
		// HandledRetryableError. But that's not what we want; we want the global
		// TxnCoordSender to do the txn restarting. So I think DistSQL should bypass

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

@andreimatei andreimatei force-pushed the distsql-retryable-err branch from 5d89c15 to 62ed249 Compare April 27, 2017 18:59
@andreimatei
Copy link
Contributor Author

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 TxnCoordSender on all the nodes but the gateway.


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…

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.

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…

Why did this need to be added here? Have we lost something on the non-test path?

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…

I think it should be txnMeta.txn.Priority. That will be updated when we learn about the current transaction from the server.

txnMeta.txn.Priority is an int; this wants a UserPriority which is a class based on which those ints get generated. I don't currently understand this flow of priorities very well...


pkg/kv/txn_coord_sender.go, line 519 at r6 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This process seems awfully convoluted, but I guess it's OK and nicely localized for now.

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…

It makes me really nervous that this is done in terms of GoError instead of ErrorDetail.

the idea is that this code (or at least the

newTxn = roachpb.PrepareTransactionForRetry(retryErr, ba.UserPriority, ba.Txn.Isolation, ba.Txn.Name)

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 pErr.GoError(). We could perhaps give DistSQL access to the ErrorDetails (if we make GoError not convert them to an InternalRetryableTxnError), but DistSQL also needs to get its hands on the pErr.Txn, and I don't know how I'd do that...

Why does this make you so nervous?


pkg/roachpb/data.go, line 973 at r6 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Which param? pri?

yes, clarified.


pkg/roachpb/errors.go, line 169 at r6 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

Why are HandledRetryableErrors turned into RetryableTxnError? HandledRetryableError implements the error interface so it looks like it should be fine to return the detail directly here.

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 force_retry function and other testing.

Note that the comment about NewRetryableTxnErrorreferred strictly to that constructor (and might have been obsolete by now). RetryableTxnError was always used broadly above KV, in SQL.

But I've gotten rid of RetryableTxnError.


pkg/roachpb/errors.go, line 444 at r6 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

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.

Well the restart has been "handled" in the sense that a Transaction proto has been prepared. But higher layers do need to act on it too - the client.Txn needs to recognize it and change state, and, in cases when the retry is not handled at the txn.Exec level, SQL needs to recognize it and potentially pass it to the SQL client.

But I've removed the canRestartTransaction interface from HandledTxnError. It wasn't really needed; layers above TxnCoordSender now only look explicitly for this error, and not other canRestart details.


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?

My TODO was about the communication between storage and KV (through pErr.Transaction), not about this error that's used above storage (so that pErr.GoError() wouldn't have to do the error munging that it's doing). So changing anything here wouldn't make me any happy wrt what I was thinking.

As to whether we could use the Transaction field here like you're suggesting - I guess we could... But I like it more this way; more explicit... I don't think a partially initialized field, that a client still has operate on, would make anything more easy or clear.
Would you be happier if I turned the 2 alternatives into a one_of set?


pkg/roachpb/errors.proto, line 314 at r6 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

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)

got rid of RetryableTxnError. I'd keep this one with Internal...


pkg/roachpb/errors.proto, line 369 at r6 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

This field doesn't appear to ever be used.

hmmm yes good catch. I've put this here because, before this change, the TxnCoordSender had this line before:
https://github.com/cockroachdb/cockroach/blob/7eb8b75/pkg/kv/txn_coord_sender.go#L853

That line seems nonsensical to me, because a) pErr.GetTxn() had already been used in relation to newTxn and, b) more importantly, client.Txn was ignoring everything but the priority from pErr when creating a new Transaction. The timestamp was left to be initialized, I guess, on the next client.Txn.Send().

So, I've now removed this field.


Comments from Reviewable

@bdarnell
Copy link
Contributor

Reviewed 19 of 19 files at r7.
Review status: all files reviewed at latest revision, 12 unresolved discussions, some commit checks failed.


pkg/kv/txn_coord_sender.go, line 516 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

txnMeta.txn.Priority is an int; this wants a UserPriority which is a class based on which those ints get generated. I don't currently understand this flow of priorities very well...

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 client.Txn (which has already stored its UserPriority) gets a handled retryable error it does txn.UpgradePriority(roachpb.MakePriority(userPriority)) on the error's roachpb.Transaction before using it.


pkg/kv/txn_coord_sender.go, line 519 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

the real problem, I think, is that the TxnCS and the client.Txn are forced to communicate through the narrow Sender interface.

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…

the idea is that this code (or at least the

newTxn = roachpb.PrepareTransactionForRetry(retryErr, ba.UserPriority, ba.Txn.Isolation, ba.Txn.Name)

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 pErr.GoError(). We could perhaps give DistSQL access to the ErrorDetails (if we make GoError not convert them to an InternalRetryableTxnError), but DistSQL also needs to get its hands on the pErr.Txn, and I don't know how I'd do that...

Why does this make you so nervous?

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 roachpb.Error.GoError, and instead change roachpb.NewError (or add new roachpb.Error constructors) that can reassemble a roachpb.Error from the pieces that distsql has. Then this code and roachpb.PrepareTransactionForRestart would work with roachpb.Errors and ErrorDetails instead of InternalRetryableTxnError.


pkg/roachpb/errors.go, line 169 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

Note that the comment about NewRetryableTxnErrorreferred strictly to that constructor (and might have been obsolete by now). RetryableTxnError was always used broadly above KV, in SQL.

But I've gotten rid of RetryableTxnError.

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 return e.GetDetail() at the end of the method.


pkg/roachpb/errors.go, line 71 at r7 (raw file):

	if intErr, ok := err.(*internalError); ok {
		*e = *(*Error)(intErr)
		// !!!

What's the plan here?


pkg/roachpb/errors.proto, line 350 at r5 (raw file):

Previously, andreimatei (Andrei Matei) 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 pErr.GoError() wouldn't have to do the error munging that it's doing). So changing anything here wouldn't make me any happy wrt what I was thinking.

As to whether we could use the Transaction field here like you're suggesting - I guess we could... But I like it more this way; more explicit... I don't think a partially initialized field, that a client still has operate on, would make anything more easy or clear.
Would you be happier if I turned the 2 alternatives into a one_of set?

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…

got rid of RetryableTxnError. I'd keep this one with Internal...

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…

hmmm yes good catch. I've put this here because, before this change, the TxnCoordSender had this line before:
https://github.com/cockroachdb/cockroach/blob/7eb8b75/pkg/kv/txn_coord_sender.go#L853

That line seems nonsensical to me, because a) pErr.GetTxn() had already been used in relation to newTxn and, b) more importantly, client.Txn was ignoring everything but the priority from pErr when creating a new Transaction. The timestamp was left to be initialized, I guess, on the next client.Txn.Send().

So, I've now removed this field.

The linked line was redundant before, but is it now? The newTxn.Update(errTxn) line is gone, and I can't find another place where the same thing happens now.

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

@andreimatei andreimatei force-pushed the distsql-retryable-err branch from 62ed249 to a2e40cc Compare April 28, 2017 02:17
@andreimatei
Copy link
Contributor Author

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…

Don't forget this.

Done.


pkg/internal/client/txn.go, line 941 at r6 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

What's the plan here?

see now pls


pkg/kv/txn_coord_sender.go, line 516 at r6 (raw file):

Previously, bdarnell (Ben Darnell) 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 client.Txn (which has already stored its UserPriority) gets a handled retryable error it does txn.UpgradePriority(roachpb.MakePriority(userPriority)) on the error's roachpb.Transaction before using it.

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…

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 roachpb.Error.GoError, and instead change roachpb.NewError (or add new roachpb.Error constructors) that can reassemble a roachpb.Error from the pieces that distsql has. Then this code and roachpb.PrepareTransactionForRestart would work with roachpb.Errors and ErrorDetails instead of InternalRetryableTxnError.

Done.


pkg/roachpb/errors.go, line 169 at r6 (raw file):

Previously, bdarnell (Ben Darnell) 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 return e.GetDetail() at the end of the method.

Done.


pkg/roachpb/errors.go, line 71 at r7 (raw file):

Previously, bdarnell (Ben Darnell) wrote…

What's the plan here?

see now pls


pkg/roachpb/errors.proto, line 314 at r6 (raw file):

Previously, bdarnell (Ben Darnell) 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.

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):
I've put the timestamp from the error in the wiped txn, and added a TODO in client.Txn about maybe not ignoring it when we initialize the txn proto on the next Send().

The linked line was redundant before, but is it now? The newTxn.Update(errTxn) line is gone, and I can't find another place where the same thing happens now.

The line newTxn.Update(errTxn) is still there - TnxCoordSender.updateState(). the "non-retryable error" cases.


Comments from Reviewable

@bdarnell
Copy link
Contributor

:lgtm_strong: (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.
Review status: all files reviewed at latest revision, 2 unresolved discussions, some commit checks failed.


pkg/roachpb/errors.proto, line 314 at r6 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

renamed to... drums... DistSQLRetryableTxnError. Cause DistSQL is the only one that's supposed to ever see it. Curious if you'll dig.

👍


Comments from Reviewable

@andreimatei andreimatei force-pushed the distsql-retryable-err branch from a2e40cc to dcf85cd Compare April 28, 2017 04:58
@andreimatei
Copy link
Contributor Author

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.
@andreimatei andreimatei force-pushed the distsql-retryable-err branch 2 times, most recently from 26f0e1b to 78648fb Compare April 28, 2017 05:18
…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.
@andreimatei andreimatei force-pushed the distsql-retryable-err branch from 78648fb to 659d611 Compare April 28, 2017 05:19
@andreimatei andreimatei merged commit 1ae28f6 into cockroachdb:master Apr 28, 2017
@andreimatei andreimatei deleted the distsql-retryable-err branch April 28, 2017 05:39
@RaduBerinde
Copy link
Member

Awesome! Great work!!

@vivekmenezes
Copy link
Contributor

vivekmenezes commented Apr 28, 2017 via email

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

5 participants