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

colexec: improve eager cancellation in parallel unordered sync #127376

Closed
wants to merge 1 commit into from

Conversation

yuzefovich
Copy link
Member

@yuzefovich yuzefovich commented Jul 17, 2024

This commit improves recently merged fix in dda8b3a. In particular, we have eager cancellation of work in the parallel unordered synchronizer for local plans, and we now do that only for subtrees that have no PUSes in them. This commit lifts that restriction by swallowing all context cancellation errors in the draining state.

The rationale for why this behavior is safe is the following:

  • if the query should result in an error, then some other error must have been propagated to the client, and this is what caused the sync to transition into the draining state in the first place. (We do replace errors for the client in one case - set DistSQLReceiver.SetError where some errors from KV have higher priority then others, but it isn't applicable here.)
  • if the query should not result in an error and should succeed, yet we have some pending context cancellation errors, then it must be the case that query execution was short-circuited (e.g. because of the LIMIT), so we can pretend the part of the execution that hit the pending error didn't actually run since clearly it wasn't necessary to compute the query result.

Note that we couldn't swallow all types of errors in the draining state (e.g. ReadWithinUncertaintyIntervalError that comes from the KV layer results in "poisoning" the txn, so we need to propagate it to the client), so we only have a single error type that we swallow.

Epic: None
Release note: None

@cockroach-teamcity
Copy link
Member

This change is Reviewable

@yuzefovich yuzefovich requested review from mgartner and michae2 July 17, 2024 18:28
@yuzefovich yuzefovich marked this pull request as ready for review July 17, 2024 18:28
@yuzefovich yuzefovich requested a review from a team as a code owner July 17, 2024 18:28
@yuzefovich
Copy link
Member Author

This is an alternative to #127076, and after discussion with Michael I'm leaning towards using this approach as well as backporting it. Curious what @mgartner thinks - does this PR on its own and the justification make sense to you?

@yuzefovich yuzefovich force-pushed the fix-sync-v2 branch 3 times, most recently from 6cc9b5b to fdc3d5e Compare July 17, 2024 19:20
Copy link
Collaborator

@michae2 michae2 left a comment

Choose a reason for hiding this comment

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

Reviewed 1 of 2 files at r1, 4 of 4 files at r2, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @mgartner and @yuzefovich)


pkg/sql/colexec/parallel_unordered_synchronizer.go line 271 at r2 (raw file):

							// synchronizer is only interested in the metadata
							// at this point.
							continue

I'm a little unsure about removing this. It seems like if the Next goroutine reads the error it could propagate out of Next as an internal error.

Copy link
Member Author

@yuzefovich yuzefovich 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 (waiting on @mgartner and @michae2)


pkg/sql/colexec/parallel_unordered_synchronizer.go line 271 at r2 (raw file):

Previously, michae2 (Michael Erickson) wrote…

I'm a little unsure about removing this. It seems like if the Next goroutine reads the error it could propagate out of Next as an internal error.

Thinking behind removal of this code is the following: the if condition can only be true when DrainMeta was called on the PUS which transitioned it into the draining state (and canceled the context in which this input goroutine is running). We only call Next and DrainMeta of the PUS from the same goroutine, so if the caller already called the latter, no more calls to Next will happen. Apart from Next we only read from errCh in DrainMeta, and now that we ignore all errors there in the draining state, this if block became redundant (i.e. we can send the error on the channel and then it'll be ignored later). In other words, we now have a more general swallowing of errors going on, so this if would only complicate things without doing anything meaningful.

Copy link
Collaborator

@michae2 michae2 left a comment

Choose a reason for hiding this comment

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

:lgtm: AFAICT this is correct, but I'm wondering if there is a way to make the DrainMeta logic a little less aggressive about silencing errors. (But if not, this is ok with me.)

Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @mgartner and @yuzefovich)


pkg/sql/colexec/parallel_unordered_synchronizer.go line 271 at r2 (raw file):

Previously, yuzefovich (Yahor Yuzefovich) wrote…

Thinking behind removal of this code is the following: the if condition can only be true when DrainMeta was called on the PUS which transitioned it into the draining state (and canceled the context in which this input goroutine is running). We only call Next and DrainMeta of the PUS from the same goroutine, so if the caller already called the latter, no more calls to Next will happen. Apart from Next we only read from errCh in DrainMeta, and now that we ignore all errors there in the draining state, this if block became redundant (i.e. we can send the error on the channel and then it'll be ignored later). In other words, we now have a more general swallowing of errors going on, so this if would only complicate things without doing anything meaningful.

ok, SGTM


pkg/sql/colexec/parallel_unordered_synchronizer.go line 431 at r2 (raw file):

			// the part of the execution that hit the pending error didn't
			// happen since clearly it wasn't necessary to compute the query
			// result.

I think this is right, but it seems pretty aggressive to always silence all errors from local flows in DrainMeta.

Could we do something like use a context.CancelCauseFunc with a specific error in the cancellation loop above, and then only silence that specific error? (But I think you were saying that some operators emit other errors on context cancelation?)

@yuzefovich yuzefovich requested a review from michae2 July 20, 2024 16:35
Copy link
Member Author

@yuzefovich yuzefovich 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 (waiting on @mgartner and @michae2)


pkg/sql/colexec/parallel_unordered_synchronizer.go line 431 at r2 (raw file):

Previously, michae2 (Michael Erickson) wrote…

I think this is right, but it seems pretty aggressive to always silence all errors from local flows in DrainMeta.

Could we do something like use a context.CancelCauseFunc with a specific error in the cancellation loop above, and then only silence that specific error? (But I think you were saying that some operators emit other errors on context cancelation?)

Hm, interesting idea. Looks like context.Cause returns the error with which the context was canceled whereas ctx.Err() still returns the context canceled error, so I'm assuming the error we'd receive in the metadata would be the latter. This mechanism would allow us to confirm that the context was canceled by the PUS eagerly and not because the flow context was canceled, but we can do this check in a different way (since we have the "flow context" at .Ctx).

In terms of some operators returning a different error on context cancellation - I only have some vague idea that it is possible (although probably unlikely in practice). For example, what happens when the KV layer observes the context cancellation? It might wrap it into a non-retryable or ambiguous error or something, I just don't know whether it happens or not. One precedent for replacing the context canceled error with something else is cancelchecker.QueryCanceledError that we return in a few places and communicate it to the client.

I'm still leaning towards swallowing everything in DrainMeta in local flows given that we don't see any flaw in this logic, and implementing this CancelCauseFunc doesn't seem to provide much benefits. Thoughts?

@michae2
Copy link
Collaborator

michae2 commented Jul 23, 2024

[summarizing discussion from Toronto] some concern that we might silence ReadWithinUncertaintyInterval errors or other errors from KV that indicate the transaction is dead, so we're thinking about going back to the first PR. Maybe we could eventually do this idea with an approved list of errors to silence?

This commit improves recently merged fix in
dda8b3a. In particular, we have eager
cancellation of work in the parallel unordered synchronizer for local
plans, and we now do that only for subtrees that have no PUSes in them.
This commit lifts that restriction by swallowing all context
cancellation errors in the draining state.

The rationale for why this behavior is safe is the following:
- if the query should result in an error, then some other error must
have been propagated to the client, and this is what caused the sync to
transition into the draining state in the first place. (We do replace
errors for the client in one case - set `DistSQLReceiver.SetError` where
some errors from KV have higher priority then others, but it isn't
applicable here.)
- if the query should not result in an error and should succeed, yet we
have some pending context cancellation errors, then it must be the case
that query execution was short-circuited (e.g. because of the LIMIT), so
we can pretend the part of the execution that hit the pending error
didn't actually run since clearly it wasn't necessary to compute the
query result.

Note that we couldn't swallow all types of errors in the draining state
(e.g. ReadWithinUncertaintyIntervalError that comes from the KV layer
results in "poisoning" the txn, so we need to propagate it to the
client), so we only have a single error type that we swallow.

Release note: None
Copy link

blathers-crl bot commented Jul 24, 2024

It looks like your PR touches production code but doesn't add or edit any test code. Did you consider adding tests to your PR?

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

@yuzefovich yuzefovich changed the title colexec: swallow errors when draining parallel unordered sync in local flow colexec: improve eager cancellation in parallel unordered sync Jul 24, 2024
Copy link
Member Author

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

Updated the change to only swallow context cancellation errors in the metadata in the draining state of the PUS. PTAL.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner and @michae2)

Copy link
Collaborator

@michae2 michae2 left a comment

Choose a reason for hiding this comment

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

Reviewed 5 of 5 files at r3, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner and @yuzefovich)


pkg/sql/colexec/parallel_unordered_synchronizer.go line 310 at r3 (raw file):

							// synchronizer is only interested in the metadata
							// at this point.
							continue

I think we also need to avoid calling sendErr with a context.Canceled because it might fill the errCh and prevent a legitimate error like ReadWithinUncertaintyIntervalError from being received by the Next goroutine.

Copy link
Member Author

@yuzefovich yuzefovich left a comment

Choose a reason for hiding this comment

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

While working on a fix to #127942, I realized why we shouldn't proceed with lifting the restriction of only allowing eager cancellation for "leaf" PUSes. Namely, if we allow eager cancellation on multiple levels, then a child PUS won't be able to distinguish between a benign eager cancellation by the parent PUS from a flow cancellation. The latter means "shutdown as quickly as possible and drop everything" whereas the former means "stop execution but collect metadata". As such, we might not properly collect the metadata in the child PUS if we allow eager cancellation in the parent PUS.

Thus, I'll go ahead and close this PR. However, I did incorporate some of the progress from here into one of the fix I'm working on. Thanks Michael for all the discussion on this one!

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @mgartner and @michae2)


pkg/sql/colexec/parallel_unordered_synchronizer.go line 310 at r3 (raw file):

Previously, michae2 (Michael Erickson) wrote…

I think we also need to avoid calling sendErr with a context.Canceled because it might fill the errCh and prevent a legitimate error like ReadWithinUncertaintyIntervalError from being received by the Next goroutine.

This if condition is only true when the PUS transitioned into draining. This means that the main goroutine will not call Next anymore; instead, it already called DrainMeta and is currently waiting for the metadata, so any remaining errors can only be propagated via batchCh and errCh won't be pushed into anymore.

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.

3 participants