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

colflow: fix the shutdown for good #74163

Merged
merged 3 commits into from
Dec 22, 2021
Merged

Conversation

yuzefovich
Copy link
Member

@yuzefovich yuzefovich commented Dec 21, 2021

Revert "Revert "colrpc: propagate the flow cancellation as ungraceful for FlowStream RPC""

This reverts commit 48dd74c.

colrpc: deflake TestOutboxInbox

In flowCtxCancel scenario there are two possible errors that might be
returned to the reader depending on the exact sequence of events:

  • QueryCanceledError is used when the flow ctx cancellation is
    observed before the stream arrived
  • wrapped context.Canceled error is used when the inbox handler
    goroutine notices the cancellation first and ungracefully shuts down
    the stream.

Previously, we assumed that only the latter could occur, and we allow
for either.

Fixes: #73962.

Release note: None

colflow: fix the shutdown for good

This commit fixes a long standing bug in the distributed vectorized
query shutdown where in case of a graceful completion of the flow on one
node, we might get an error on another node resulting in the ungraceful
termination of the query. This was caused by the fact that on remote
nodes the last outbox to exit would cancel the flow context; however,
when instantiating FlowStream RPC the outboxes used a child context of
the flow context, so that "graceful" cancellation of the flow context
would cause the inbox to get an ungraceful termination of the gRPC
stream. As a result, the whole query could get "context canceled" error.

I believe this bug was introduced by me over two years ago because
I didn't fully understand how the shutdown should work, and in
particular I was missing that when an inbox observes the flow context
cancellation, it should terminate the FlowStream RPC ungracefully in
order to propagate the ungracefullness to the other side of the stream.
This shortcoming was fixed in the previous commit.

Another possible bug was caused by the outbox canceling its own context
in case of a graceful shutdown. As mentioned above, FlowStream RPC was
issued using the outbox context, so there was a possibility of a race
between CloseSend call being delivered to the inbox (graceful
termination) and the context of the RPC being canceled (ungraceful
termination).

Both of these problems are now fixed, and the shutdown protocol now is as
follows:

  • on the gateway node we keep on canceling the flow context at the very
    end of the query execution. It doesn't matter whether the query resulted
    in an error or not, and doing so allows us to ensure that everything
    exits on the gateway node. This behavior is already present.
  • due to the fix in a previous commit, that flow context cancellation
    terminates ungracefully all still open gRPC streams for FlowStream RPC
    for which the gateway node is the inbox host.
  • the outboxes on the remote nodes get the ungraceful termination and
    cancel the flow context of their hosts. This, in turn, would trigger
    propagation of the ungraceful termination on other gRPC streams, etc.
  • whenever an outbox exits gracefully, it cancels its own context, but
    the gRPC stream uses the flow context, so the stream is still alive.
    I debated a bit whether we want to keep this outbox context cancellation
    in case of a graceful completion and decided to keep it to minimize the
    scope of changes.

Fixes: #73425.
Fixes: #73966.
Fixes: #73972.

Release note (bug fix): Previously, CockroachDB could return a spurious
"context canceled" error for a query that actually succeeded in
extremely rare cases, and this has now been fixed.

In `flowCtxCancel` scenario there are two possible errors that might be
returned to the reader depending on the exact sequence of events:
- `QueryCanceledError` is used when the flow ctx cancellation is
observed before the stream arrived
- wrapped `context.Canceled` error is used when the inbox handler
goroutine notices the cancellation first and ungracefully shuts down
the stream.

Previously, we assumed that only the latter could occur, and we allow
for either.

Release note: None
This commit fixes a long standing bug in the distributed vectorized
query shutdown where in case of a graceful completion of the flow on one
node, we might get an error on another node resulting in the ungraceful
termination of the query. This was caused by the fact that on remote
nodes the last outbox to exit would cancel the flow context; however,
when instantiating `FlowStream` RPC the outboxes used a child context of
the flow context, so that "graceful" cancellation of the flow context
would cause the inbox to get an ungraceful termination of the gRPC
stream. As a result, the whole query could get "context canceled" error.

I believe this bug was introduced by me over two years ago because
I didn't fully understand how the shutdown should work, and in
particular I was missing that when an inbox observes the flow context
cancellation, it should terminate the `FlowStream` RPC ungracefully in
order to propagate the ungracefullness to the other side of the stream.
This shortcoming was fixed in the previous commit.

Another possible bug was caused by the outbox canceling its own context
in case of a graceful shutdown. As mentioned above, `FlowStream` RPC was
issued using the outbox context, so there was a possibility of a race
between `CloseSend` call being delivered to the inbox (graceful
termination) and the context of the RPC being canceled (ungraceful
termination).

Both of these problems are now fixed, and the shutdown protocol now is as
follows:
- on the gateway node we keep on canceling the flow context at the very
end of the query execution. It doesn't matter whether the query resulted
in an error or not, and doing so allows us to ensure that everything
exits on the gateway node. This behavior is already present.
- due to the fix in a previous commit, that flow context cancellation
terminates ungracefully all still open gRPC streams for `FlowStream` RPC
for which the gateway node is the inbox host.
- the outboxes on the remote nodes get the ungraceful termination and
cancel the flow context of their hosts. This, in turn, would trigger
propagation of the ungraceful termination on other gRPC streams, etc.
- whenever an outbox exits gracefully, it cancels its own context, but
the gRPC stream uses the flow context, so the stream is still alive.
I debated a bit whether we want to keep this outbox context cancellation
in case of a graceful completion and decided to keep it to minimize the
scope of changes.

Release note (bug fix): Previously, CockroachDB could return a spurious
"context canceled" error for a query that actually succeeded in
extremely rare cases, and this has now been fixed.
@yuzefovich yuzefovich requested review from jordanlewis and a team December 21, 2021 18:31
@cockroach-teamcity
Copy link
Member

This change is Reviewable

@yuzefovich
Copy link
Member Author

The first commit is exactly #73887, and I reduced the scope of changes in the third commit (comparing to #74121), so I'm quite pleased with this change overall when thinking about the backport.

@yuzefovich yuzefovich requested a review from cucaroach December 21, 2021 19:19
@yuzefovich
Copy link
Member Author

I've stressed distsql_union logic test for a while with no failures:

11688 runs so far, 0 failures, over 2h48m45s

so I think we're good.

@jordanlewis @cucaroach I'd like to merge this today if possible so that we get a nightly CI run to see whether there is any fallout before the holidays.

Copy link
Contributor

@cucaroach cucaroach left a comment

Choose a reason for hiding this comment

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

Its unfortunate that you have to submit to master to get your change run through nightly, maybe we wouldn't have so many test failures if that wasn't the case. Anyways I reviewed it but only superficially, understanding this stuff better will have to wait till next year!

Reviewed 3 of 6 files at r1, 2 of 2 files at r2, 3 of 3 files at r3, all commit messages.
Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (waiting on @jordanlewis)

@yuzefovich
Copy link
Member Author

I'm relatively confident that this change should not have any fallout, but I'm very interested in the SQLLite logic tests that are run nightly. I think they are pretty expensive, so it would not be really feasible to include it as part of CI for each PR.

I could, however, manually kick off some nightly builds on my current branch without merging it to master, but there is a bit of urgency on merging this - we have a customer on 21.1 that is blocked by this, and I want to have several days worth of nightly runs for more confidence before merging this change to 21.1.

Unless something else comes up, I plan to write the draft of the tech note about the distributed query shutdown tomorrow, and we can chat more after the holidays :)

TFTR!

bors r+

@craig
Copy link
Contributor

craig bot commented Dec 22, 2021

Build succeeded:

@craig craig bot merged commit 3a94842 into cockroachdb:master Dec 22, 2021
@yuzefovich yuzefovich deleted the fix-flow-ctx branch December 22, 2021 02:05
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants