-
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
distsql: tpch query 7 results in context cancelled
error
#13989
Comments
This issue is caused by a flow not being set up in time and the corresponding inbound stream timing out (see This investigation will continue with the changes in #13570 rather than on master due to the size of the PR and the imminent merge. Note: A simpler query to run to reproduce the same result is the inner |
I've managed to reproduce a "context cancelled" error in another test that I'm looking at. Hopefully it's the same cause. Who knew we're so good at cancelling contexts? |
The "contex cancelled" error comes from a the server side of a grpc streaming connection returning from the RPC handler. The client's Anyway, @asubiotto, if you see this error again, please give me a stack trace. |
Regardless, the code that crashes in if is.timedOut {
is.receiver.Close(errors.Errorf("inbound stream timed out"))
} |
Close() is already called when timedOut is set. fixes cockroachdb#13989
Yup, I'm fixing that.
…On Sun, Mar 12, 2017 at 7:42 PM, RaduBerinde ***@***.***> wrote:
Regardless, the code that crashes in finishInboundStreamLocked seems
bogus; I think it just needs to be removed (we already close the receiver
when we set timedOut):
if is.timedOut {
is.receiver.Close(errors.Errorf("inbound stream timed out"))
}
—
You are receiving this because you commented.
Reply to this email directly, view it on GitHub
<#13989 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AAXBcU_kH8-b196m9xJvq0bbYEBA9rBTks5rlIL0gaJpZM4MXBgy>
.
|
I've already sent a fix for that
On Mon, Mar 13, 2017 at 12:34 AM Andrei Matei <[email protected]>
wrote:
… Yup, I'm fixing that.
On Sun, Mar 12, 2017 at 7:42 PM, RaduBerinde ***@***.***>
wrote:
> Regardless, the code that crashes in finishInboundStreamLocked seems
> bogus; I think it just needs to be removed (we already close the receiver
> when we set timedOut):
>
> if is.timedOut {
> is.receiver.Close(errors.Errorf("inbound stream timed out"))
> }
>
> —
> You are receiving this because you commented.
> Reply to this email directly, view it on GitHub
> <
#13989 (comment)
>,
> or mute the thread
> <
https://github.com/notifications/unsubscribe-auth/AAXBcU_kH8-b196m9xJvq0bbYEBA9rBTks5rlIL0gaJpZM4MXBgy
>
> .
>
—
You are receiving this because you are subscribed to this thread.
Reply to this email directly, view it on GitHub
<#13989 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/ALOpBDId2lq7TDxU7Ejw4u3XyUMctXzCks5rlMdogaJpZM4MXBgy>
.
|
Played around with gRPC a bit. The "context cancelled" errors that I was able to reproduce are received by a server that tries to access the stream after the rpc handler has returned (so a reference to the stream has been saved and used concurrently with the handler). But I'm not sure how that would happen with our production code, since servers ( So I'm not sure how the "context cancelled" error happened. When it happens again we need more info about which component is involved. |
We didn't have any tests exercising a stream failing to connect within the registry's timeout.
We didn't have any tests exercising a stream failing to connect within the registry's timeout. I've changed the fr.ConnectInbound interface slightly - made it return a RowReceiver explicitly so that callers don't access the inboundStreamInfo's fields without the flow registry lock - which is documented to be required.
We didn't have any tests exercising a stream failing to connect within the registry's timeout. I've changed the fr.ConnectInbound interface slightly - made it return a RowReceiver explicitly so that callers don't access the inboundStreamInfo's fields without the flow registry lock - which is documented to be required.
I'm certainly not seeing this any more on the nightly tests I'm running with schema changes. @asubiotto you were able to reproduce this. Can you still do? |
He's on vacation. @arjunravinarayan do you see this issue when running your TPC-H load generator? |
I'll report back later today. I'm setting up omega to run TPC h query load
continuously so we will be stressing this query shortly.
|
Yes, I am still seeing this when running TPC-H query 7 on cluster omega (6 nodes, running beta candidate
|
Apologies for taking so long, there were many bugs encountered along the way, including cockroachdb/loadgen#34, cockroachdb/loadgen#35, and some other discoveries in backup/restore. |
Great! |
context cancelled
error
@arjunravinarayan thanks! Glad you both are on it! |
I dug a little bit. I could repro this "context canceled" error on my laptop with 2 local nodes, although it doesn't always happen. Based on the logs pasted below (taken from a node different from the gateway), this seems to be caused by the heartbeating that we do on gRPC connections to see if the connection is still alive. One of these heartbeats times out and so we close the gprc connection on the client side (the client here corresponds to the FlowStream RPC - so it's the data producer). This closing of the connection translates into a server-side "context canceled" error. It's interesting that the server receives the "context canceled" information (presumably a TPC reset packet) very soon after the client closes the connection, so communication at some level is not broken. I have a theory that this is caused by bad multiplexing by gRPC of different calls on top of a single HTTP2 connection. Perhaps pushing a lot of data on a streaming RPC at the same time as trying to do a small heartbeat RPC doesn't guarantee any QoS for the heartbeat. I'll step in the lab to test this out.
|
Ah, I see. Yeah, that argues for a higher ratio of connection quota to per-stream quota (and maybe a lower value for the per-stream quota, at least for these streams). At the lowest level HTTP/2 gives you a lot of flexibility about how to handle these quotas (for example, the connection quota need not be static; you could increase it whenever you find a lot of data buffered in streams with ordered synchronizers), but a lot of the subtleties are lost by the time you get up to the application layer. |
One mystery has been solved - we use @petermattis' fork of grpc which makes the conn window size = stream window size = 2MB. I checked for this fork before but I think I must have looked at the wrong file cause grpc is sub-vendored multiple times in our tree. There is some WIP on grpc about dynamic connection window sizes that Tamir showed me: grpc/grpc-go#1126 Ben, I'm not sure how serious you were about exploring different schemes. I think we do want to be cognizant of bandwidth limits and not thrash, but I don't think we want to use grpc flow control for that. We throttle replication of new ranges in/out of a node, we have some (primitive) admission control for DistSQL queries - I think these are better layers at which to deal with bandwidth. No? |
Switch from Peter's fork of grpc to my own fork, since Peter's out of office. My fork is https://github.com/grpc/grpc-go head + Peter's one commit increasing the per-stream flow-control window. The motivation of syncing to newer gRPC is to include grpc/grpc-go#993 for helping with cockroachdb/cockroach#13989 - we will move to gRPC internal heartbeats instead of using our own connection heartbeats. Other dep updates: - the Lightstep update is a single commit that seems innocuous
Switch from Peter's fork of grpc to my own fork, since Peter's out of office. My fork is https://github.com/grpc/grpc-go head + Peter's one commit increasing the per-stream flow-control window. The motivation of syncing to newer gRPC is to include grpc/grpc-go#993 for helping with cockroachdb#13989 - we will move to gRPC internal heartbeats instead of using our own connection heartbeats in a future cockroach commit. Other dep updates: - the Lightstep update is a single commit that seems innocuous
Ah. Setting those two to equal is a bad idea since it lets one operation (like a snapshot) monopolize the whole connection.
it lets you discover the BDP at runtime and use that to set the flow control limits instead of hard-coding a constant like we've done so far.
I don't want to do anything too fancy here (i.e. I want to avoid dropping down to low-level HTTP/2 stuff). We should definitely tweak the numbers to set the connection limit higher than the stream limit (e.g. to 16x like it was before). If it's still an issue after that then we can look at making the per-connection limit unbounded and relying on other sorts of flow control |
- use gRPC Keepalives for detecting dead connections Before this patch, the rpc.Context would perform heartbeats (a dedicated tRPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. Heartbeats stay for the other current uses - clock skew detecting and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown. Fixes cockroachdb#13989 - make heartbeats and clock skew detecting optional. Not all users of rpc.Context need this functionality.
- use gRPC Keepalives for detecting dead connections Before this patch, the rpc.Context would perform heartbeats (a dedicated tRPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. Heartbeats stay for the other current uses - clock skew detecting and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown. Fixes cockroachdb#13989 - make heartbeats and clock skew detecting optional. Not all users of rpc.Context need this functionality.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
Fixes cockroachdb#13989 Before this patch, the rpc.Context would perform heartbeats (a dedicated RPC) to see if a connection is healthy. If the heartbeats failed, the connection was closed (causing in-flight RPCs to fail) and the node was marked as unhealthy. These heartbeats, being regular RPCs, were subject to gRPC's flow control. This means that they were easily blocked by other large RPCs, which meant they were too feeble. In particular, they were easily blocked by large DistSQL streams. This patch moves to using gRPC's internal HTTP2 ping frames for checking conn health. These are not subject to flow control. The grpc transport-level connection is closed when they fail (and so in-flight RPCs still fail), but otherwise gRPC reconnects transparently. Heartbeats stay for the other current uses - clock skew detection and node health marking. Marking a node as unhealthy is debatable, give the shortcomings of these RPCs. However, this marking currently doesn't have big consequences - it only affects the order in which replicas are tried when a leaseholder is unknown.
When running (execution plan)
against a 3-node local cluster sometimes crashes all three servers with
And sometimes only crashes the gateway server with
The text was updated successfully, but these errors were encountered: