-
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
release-21.1: colrpc: propagate the flow cancellation as ungraceful for FlowStream RPC #73959
release-21.1: colrpc: propagate the flow cancellation as ungraceful for FlowStream RPC #73959
Conversation
Thanks for opening a backport. Please check the backport criteria before merging:
If some of the basic criteria cannot be satisfied, ensure that the exceptional criteria are satisfied within.
Add a brief release justification to the body of your PR to justify this backport. Some other things to consider:
|
This wasn't a clean backport but wasn't too bad either. |
6a64d84
to
6d1ba3b
Compare
This commit fixes an oversight in the cancellation protocol of the vectorized inbox/outbox communication. Previously, when the flow context of the inbox host has been canceled (indicating that the whole query should be canceled) we would propagate it as a graceful completion of the `FlowStream` RPC which would result in the outbox cancelling only its own subtree on the remote node. However, what we ought to do is to propagate such cancellation as the ungraceful RPC completion so that the outbox would also cancel the flow context of its own host. In some rare cases the old behavior could result in some flows being stuck forever (until a node is restarted) because they would get blocked on producing the data when their consumer has already exited. The behavior in this fix is what we already have in place for the row-by-row engine (see `processInboundStreamHelper` in `flowinfra/inbound.go`). Release note (bug fix): The bug with the ungraceful shutdown of the distributed queries in some rare cases has been fixed. "Ungraceful" here means because of the `statement_timeout` (most likely) or because a node crashed.
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.
6d1ba3b
to
83e644d
Compare
I think this is now safe to merge since there hasn't been any new fallout and the reasons for why we saw some failures just with the first commit are understood. PTAL @rytaft @jordanlewis |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Reviewed 4 of 7 files at r1, 1 of 2 files at r2, 5 of 5 files at r5, 2 of 2 files at r6, 3 of 3 files at r7, all commit messages.
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @jordanlewis)
Backport 1/1 commits from #73887.
Backport 2/3 commits from #74163.
/cc @cockroachdb/release
colrpc: propagate the flow cancellation as ungraceful for FlowStream RPC
This commit fixes an oversight in the cancellation protocol of the
vectorized inbox/outbox communication. Previously, when the flow context
of the inbox host has been canceled (indicating that the whole query
should be canceled) we would propagate it as a graceful completion of
the
FlowStream
RPC which would result in the outbox cancelling onlyits own subtree on the remote node. However, what we ought to do is to
propagate such cancellation as the ungraceful RPC completion so that the
outbox would also cancel the flow context of its own host.
In some rare cases the old behavior could result in some flows being
stuck forever (until a node is restarted) because they would get blocked
on producing the data when their consumer has already exited.
The behavior in this fix is what we already have in place for the
row-by-row engine (see
processInboundStreamHelper
inflowinfra/inbound.go
).Fixes: https://github.com/cockroachlabs/support/issues/1326.
Fixes: #72445.
Release note (bug fix): The bug with the ungraceful shutdown of the
distributed queries in some rare cases has been fixed. "Ungraceful" here
means because of the
statement_timeout
(most likely) or because a nodecrashed.
colrpc: deflake TestOutboxInbox
In
flowCtxCancel
scenario there are two possible errors that might bereturned to the reader depending on the exact sequence of events:
QueryCanceledError
is used when the flow ctx cancellation isobserved before the stream arrived
context.Canceled
error is used when the inbox handlergoroutine 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
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 ofthe 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 inorder 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 wasissued using the outbox context, so there was a possibility of a race
between
CloseSend
call being delivered to the inbox (gracefultermination) 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:
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.
terminates ungracefully all still open gRPC streams for
FlowStream
RPCfor which the gateway node is the inbox host.
cancel the flow context of their hosts. This, in turn, would trigger
propagation of the ungraceful termination on other gRPC streams, etc.
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.