Skip to content

distsqlrun,kv,client: don't create flows in aborted txns#29455

Merged
craig[bot] merged 6 commits intocockroachdb:masterfrom
andreimatei:flow-in-aborted-txn
Sep 5, 2018
Merged

distsqlrun,kv,client: don't create flows in aborted txns#29455
craig[bot] merged 6 commits intocockroachdb:masterfrom
andreimatei:flow-in-aborted-txn

Conversation

@andreimatei
Copy link
Copy Markdown
Contributor

@andreimatei andreimatei commented Aug 31, 2018

Before this patch, we had a race between a heartbeat find out (async)
that a txn has been aborted and a client using the txn to create DistSQL
flows. We could end up creating flows in aborted txns, which are going
to have a bad time: at the moment, leaf TxnCoordSenders don't react well
to what appears to be an async abort: tcs.maybeRejectClientLocked()
returns a HandledRetryableError, but the DistSQL infrastructure only
expects raw retriable errors - so the HandledRetryableError was
ironically losing its "retryable" character and was making it to a
sql client with the wrong pgwire code.
This patch resolves the situation by atomically checking that the txn is
still good (i.e. status==PENDING) before extracting its metadata that is
sent to leaves.

Fixes #28898
Fixes #29271

Release note: Fix an issue where, under severe load, clients were
sometimes receiving retryable errors with a non-retryable error code (a
client would get an error with the message "HandledRetryableError: ..."
but an internal error code instead of the expected retryable error
code).

@andreimatei andreimatei requested review from a team August 31, 2018 23:05
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

@andreimatei
Copy link
Copy Markdown
Contributor Author

cc @jordanlewis
Still have to write a test, and I'll add some more protection at different levels for trapping leaves created with aborted txns, but something like this.

Copy link
Copy Markdown
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 14 of 14 files at r1, 10 of 10 files at r2.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/internal/client/sender.go, line 109 at r2 (raw file):

	// AugmentMeta().
	//
	// If AnyTxnStatus is passed, than this function never returns errors.

s/than/then/


pkg/sql/distsql_running.go, line 138 at r2 (raw file):

		// If the plan is not local, we will have to set up leaf txns using the
		// txnCoordMeta.
		meta, err := txn.GetStrippedTxnCoordMeta(ctx, client.OnlyPending)

Mind defending why this approach is better than just checking meta.Status here and not complicating every other caller of GetStrippedTxnCoordMeta?

Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/internal/client/sender.go, line 109 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

s/than/then/

Done.


pkg/sql/distsql_running.go, line 138 at r2 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Mind defending why this approach is better than just checking meta.Status here and not complicating every other caller of GetStrippedTxnCoordMeta?

As we were discussing, it's cause we want the TCS to be in charge of producing the right error.
Check out the API now - I've moved the "stripped" flavor elsewhere, left the txn with two flavors - one produces errors, one doesn't - and now the TxnStatusOpt is just used by the TCS API, not by the Txn (which is more "client" facing).

@andreimatei andreimatei force-pushed the flow-in-aborted-txn branch 3 times, most recently from 7f0b70f to b2b0cfc Compare September 5, 2018 17:55
Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

wrote an integration test. PTAL

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained

... to a standard callback. Will be used marginally in the next commit.

Release note: None
... with a pair of methods on the TxnCoordMeta proto.
This is in preparation of txn.GetTxnCoordMeta() being split into two
flavors in the next commit.

Release note: None
@andreimatei andreimatei force-pushed the flow-in-aborted-txn branch 2 times, most recently from 60d97c9 to 92b6676 Compare September 5, 2018 18:31
@andreimatei andreimatei requested a review from a team September 5, 2018 18:55
@andreimatei andreimatei requested a review from a team as a code owner September 5, 2018 20:33
@andreimatei andreimatei requested a review from a team September 5, 2018 20:33
Copy link
Copy Markdown
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewed 11 of 11 files at r3, 6 of 6 files at r4, 17 of 17 files at r5, 1 of 1 files at r6, 2 of 2 files at r7, 15 of 35 files at r8.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/internal/client/txn.go, line 138 at r7 (raw file):

		log.Fatalf(context.TODO(), "attempting to create txn with nil db for Transaction: %s", meta.Txn)
	}
	if meta.Txn.Status != roachpb.PENDING {

Careful. What about mixed version clusters?


pkg/sql/integration_test.go, line 1 at r5 (raw file):

// Copyright 2018 The Cockroach Authors.

sql/integration_test.go is a useless file name. If this doesn't belong in any other test file then name this something descriptive.


pkg/sql/integration_test.go, line 49 at r5 (raw file):

// in it. We're careful to not use the transaction for anything but running the
// plan; planning will be performed outside of the transaction.
func TestDistSQLRunningInAbortedTxn(t *testing.T) {

Nice test!


pkg/util/tracing/test_utils.go, line 28 at r5 (raw file):

		spMsg := ""
		for _, l := range recSp.Logs {
			for _, f := range l.Fields {

Why is this needed? Do we expect callers to match against these fields? If so, should we explain this format in the function comment?

Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/internal/client/txn.go, line 138 at r7 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Careful. What about mixed version clusters?

but I've also added protection in DistSQLServer, a layer above. So DistSQL can't get here with an aborted txn, and that should be the only case where this happened.


pkg/sql/integration_test.go, line 1 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

sql/integration_test.go is a useless file name. If this doesn't belong in any other test file then name this something descriptive.

It's not useless. It's the most descriptive I can come up with. I'm not gonna name a file just for one test. You can imagine other "integration tests" coming here. There's also precedent in kv/integration_test.go :)


pkg/sql/integration_test.go, line 49 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Nice test!

Done.


pkg/util/tracing/test_utils.go, line 28 at r5 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

Why is this needed? Do we expect callers to match against these fields? If so, should we explain this format in the function comment?

well the log message itself is one of these fields

Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/sql/integration_test.go, line 1 at r5 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

It's not useless. It's the most descriptive I can come up with. I'm not gonna name a file just for one test. You can imagine other "integration tests" coming here. There's also precedent in kv/integration_test.go :)

well, I guess this didn't technically end up being an "integration test" after all, it's mostly a test for the DistSQLPlanner.Run() method. Let me see about finding another home.

Before this patch, we had a race between a heartbeat find out (async)
that a txn has been aborted and a client using the txn to create DistSQL
flows. We could end up creating flows in aborted txns, which are going
to have a bad time: at the moment, leaf TxnCoordSenders don't react well
to what appears to be an async abort: tcs.maybeRejectClientLocked()
returns a HandledRetryableError, but the DistSQL infrastructure only
expects raw retriable errors - so the HandledRetryableError was
ironically losing its "retryable" character and was making it to a
sql client with the wrong pgwire code.
This patch resolves the situation by atomically checking that the txn is
still good (i.e. status==PENDING) before extracting its metadata that is
sent to leaves.

Fixes cockroachdb#28898
Fixes cockroachdb#29271

Release note (bug fix): Fix an issue where, under severe load, clients
were sometimes receiving retryable errors with a non-retryable error
code (a client would get an error with the message
"HandledRetryableError: ..." but an internal error code instead of the
expected retryable error code).
... now that the gateway is not trying to create them any more.

Release note: None
Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained


pkg/sql/integration_test.go, line 1 at r5 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

well, I guess this didn't technically end up being an "integration test" after all, it's mostly a test for the DistSQLPlanner.Run() method. Let me see about finding another home.

moved

Copy link
Copy Markdown
Contributor

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm:

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale)


pkg/internal/client/txn.go, line 138 at r7 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

but I've also added protection in DistSQLServer, a layer above. So DistSQL can't get here with an aborted txn, and that should be the only case where this happened.

👍 just wanted to make sure you had thought about this.


pkg/sql/distsqlrun/server.go, line 330 at r12 (raw file):

	if meta := req.TxnCoordMeta; meta != nil {
		if !localState.IsLocal {
			if meta.Txn.Status != roachpb.PENDING {

nit: Maybe leave a comment that this can be removed in 2.2, unless you think we should always keep it.


pkg/sql/integration_test.go, line 1 at r5 (raw file):

Previously, andreimatei (Andrei Matei) wrote…

moved

Much better.

Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained


pkg/sql/distsqlrun/server.go, line 330 at r12 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

nit: Maybe leave a comment that this can be removed in 2.2, unless you think we should always keep it.

I think we should always keep it. It's pretty basic sanity and acts as documentation too.

Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained


pkg/internal/client/txn.go, line 138 at r7 (raw file):

Previously, nvanbenschoten (Nathan VanBenschoten) wrote…

👍 just wanted to make sure you had thought about this.

btw, I considered having this return an error instead of crash, but didn't do it as the function is too commonly called and it's not worth introducing the error handling I thought

Copy link
Copy Markdown
Contributor Author

@andreimatei andreimatei left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

bors r+

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained

craig bot pushed a commit that referenced this pull request Sep 5, 2018
29455: distsqlrun,kv,client: don't create flows in aborted txns r=andreimatei a=andreimatei

Before this patch, we had a race between a heartbeat find out (async)
that a txn has been aborted and a client using the txn to create DistSQL
flows. We could end up creating flows in aborted txns, which are going
to have a bad time: at the moment, leaf TxnCoordSenders don't react well
to what appears to be an async abort: tcs.maybeRejectClientLocked()
returns a HandledRetryableError, but the DistSQL infrastructure only
expects raw retriable errors - so the HandledRetryableError was
ironically losing its "retryable" character and was making it to a
sql client with the wrong pgwire code.
This patch resolves the situation by atomically checking that the txn is
still good (i.e. status==PENDING) before extracting its metadata that is
sent to leaves.

Fixes #28898
Fixes #29271

Release note: Fix an issue where, under severe load, clients were
sometimes receiving retryable errors with a non-retryable error code (a
client would get an error with the message "HandledRetryableError: ..."
but an internal error code instead of the expected retryable error
code).

Co-authored-by: Andrei Matei <andrei@cockroachlabs.com>
@craig
Copy link
Copy Markdown
Contributor

craig bot commented Sep 5, 2018

Build succeeded

@craig craig bot merged commit adb01ee into cockroachdb:master Sep 5, 2018
@andreimatei andreimatei deleted the flow-in-aborted-txn branch September 5, 2018 22:35
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.

teamcity: failed tests on master: testrace/TestChangefeedTimestamps, test/TestChangefeedTimestamps sql: tpcc returns retriable errors to the user

3 participants