-
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: query failures with no inbound stream connection error #31361
Comments
I think @nvanbenschoten claimed that he can repro this reliably on a geo-distributed cluster. |
Yeah, I was trying to simulate that with the network latencies. Trying to reproduce with some extra instrumentation but it might be worth it to set up what @nvanbenschoten had exactly. |
Nothing came up today with the instrumented binary. Will attempt to reproduce @nvanbenschoten's setup tomorrow. One interesting thing that jumped out to me is that if there are no workers from the 16 available (with high network latencies this is very probable), the main connection goroutine will send out the |
I haven't been able to set up a multi-regional cluster because I'm having issues restoring tpcc 5k (due to #31172). Hopefully will do so as soon as that's fixed. |
Restore has completed, running tpcc-5k on a 12-node geo-distributed cluster. No repro so far. Verified that
immediately fails the workload. @nvanbenschoten, any further guidance you can provide? I remember you said this error occurred relatively quickly |
@asubiotto mind posting the command you're using to run the load? |
Running
|
I think those regions are wrong. Take a look at
|
Thanks for noticing that! They were definitely off. I tried again today after correcting the topology but no failure. Will try again on Monday. |
Got a reproduction on
Node 6 (
But for some reason the node 10 never received a Adding more logging in suspicious areas and will try to reproduce while I attempt to puzzle this out. Additional information: node 10 is in |
Great to see this reproducing! |
Another failure but this time with a different profile. Node 8 sets up a flow:
Node 9 receives it around the same time:
Node 1 receives it 19s later:
The gateway is stuck on waiting for the RPC to be received by node 1. In this time, node 9 attempts to set up a stream, waits 10 seconds, and gives up. Let’s look at a different failure (same run). The interesting thing here is that we’re not too far apart in time. Note that the failure of the previous load generator does not affect the load in this different region. This is the failure profile of a flow that looks very similar to what I posted in my previous comment. Node 1 attempts to set up a flow:
Node 12 gets the setup message fine and apparently sends back a FlowStream message:
But Node 1 never seems to get this message. No error is returned from FlowStream, otherwise node 12 would have logged this error. If no error occurs, FlowStream on the receiving side logs something like:
The weird part is that nothing of the sort was logged, implying that the initial
Albeit slowly. Both these examples seem to point to something fishy going on at the grpc/network level. @nvanbenschoten, @andreimatei, do you have any suggestions as to how to proceed with the debugging/offer more insight into this part of the stack? Looking at Put up my branch in https://github.com/asubiotto/cockroach/tree/timeout |
My only suggestion regarding gRPC itself is what I was telling you about the other day - to try digging into the connection window management code in grpc and see if new HTTP2 streams are being starved for connection window. Although I guess if new streams couldn't be established, then the cluster as a whole would have bigger issues... |
Found one "bug". It seems that the outbox doesn't retry I've been able to reproduce the cases above with a bit more instrumentation and it does seem like we're hitting some issue with flow control. Notably, one load generator fails with:
Which is telling us that the connection closed with a there seem to be a couple of limits. I've also been looking at a blocking limit called So knowing there is an issue with flow control, I initially thought that we might not be closing streams well in general. As this commit states (grpc/grpc-go@365770f), calling cockroach/pkg/storage/raft_transport.go Line 640 in 3483037
So I added more functionality to our current RPC StatsHandler to count the number of Active RPCs through stats.Begin and stats.End . It seems like the number of active RPCs stays pretty close to 0 (second over second) so this seems less likely to be the issue, although it might be more interesting to see if we can actually get some insight into the data that is clogging up grpc (and if it is doing so unexpectedly). More reading related to closing streams: grpc/grpc-go#2015.
Having RPC tracing on in this cluster did seem to show some weird stuff. Slowly but surely, we seemed to be having longer and longer RPC durations. Most notably in @nvanbenschoten, @bdarnell are the This is pretty much where I'm at now, there is no other new information. The next step is to see whether I can find a way to keep track of how much data is read/written from a certain stream and see if I can associate that with a grpc method name. |
The RaftMessageBatch streams are long lived, so this is likely a red
herring.
|
But I believe this traces each |
Oh, that would be unexpected. The items sent on that stream are batches of Raft requests. In regular operation you don't expect very large blobs there, though during import/restore I don't doubt it could happen because we submit ~22mb Raft commands. |
Yes, thanks for reminding me! It's setting this line to true (although I believe true is the default): Line 57 in 3483037
The comment mentions the issue about the high memory usage: grpc/grpc-go#695 The rpc requests then show up in |
Something that jumps out to me (although it's just speculation) is that in: cockroach/pkg/storage/raft_transport.go Lines 428 to 443 in 3483037
We seem to be calling cockroach/pkg/storage/raft_transport.go Line 440 in 3483037
Which looks like it has the potential to take a fair amount of time. This is time not spent Recv ing so this could potentially result in a back up if messages are being sent quicker than they are being processed?
|
cockroach/pkg/storage/store.go Lines 3580 to 3656 in 3678d85
|
The more interesting direction is HandleRaftRequest cockroach/pkg/storage/raft_transport.go Line 341 in 3483037
|
My reading of this is that |
@petermattis that's a good point... @tbg I added some log lines for > 10ms handle raft requests:
And it seems like it's printing out pretty consistently:
Although the cluster is running fine for now. This definitely surprises me as that's a really long time for |
Before this PR we would sometimes call in to the nodedialer with a canceled context and inadvertently end up recording a failure into the circuit breaker. This behavior led to some amount of noise when debugging cockroachdb#31361. Release note: None
The circuit breaker I believe is something of a red herring. We were tripping it in cases where we should not be because we were not first checking for context cancellation (see #34026). The description of this issue cites that #27746 is not the cause of the issue but I question how that conclusion was reached. In my reproduction which follows from @asubiotto's branch based on Nathan's configuration I was able to reliably reproduce the failure but observed with vmodule logging on that we are indeed hitting the queue limit (even though the number of flows active according to the admin UI never seemed to be 500). How was it determined that we were not hitting the limit? I've seen a number of
My immediate plan is to add several new metrics to give us visibility into how many flows are running, how many flows are queued, and what does the distribution of wait time for queued flows look like. Such a change should be safe for back porting and can inform clients as to when they should increase Longer term we should come up for a more graceful handling of overload in situations where flows are queued for too long before starting. Ideally we'd have some mechanism to reject flows from being queued at all which would propagate to the gateway and lead to flow teardown everywhere. |
Thanks for looking into this. Seems promising. Just to double check, have you tried without vmodule but with targeted logging? I'm asking because vmodule can make the logging path slow enough to change global behavior. I hope this isn't relevant here, though. |
I'll reproduce with no vmodule logging with these new metrics. One concern I have is that my "reproduction" was #27746 and @asubiotto was seeing something different but I'm not sure I'd tell. |
34026: rpc/nodedialer: check context for cancellation before dialing r=ajwerner a=ajwerner Before this PR we would sometimes call in to the nodedialer with a canceled context and inadvertently end up recording a failure into the circuit breaker. This behavior led to some amount of noise when debugging #31361. Release note: None Co-authored-by: Andrew Werner <[email protected]>
Thanks for looking into this @ajwerner. I initially came to the conclusion that this wasn’t #27746 only due to the number of flows shown in the admin UI. However, since then, I added logging around the GRPC calls that show unexpectedly long times between a send and a receive of a If the issue was a flow scheduler backup, we would see these remote nodes receive the |
Before this PR we would sometimes call in to the nodedialer with a canceled context and inadvertently end up recording a failure into the circuit breaker. This behavior led to some amount of noise when debugging cockroachdb#31361. Release note: None
Attempted to reproduce this yesterday with @ajwerner with an updated master (with the circuitbreaker changes) and although the load generator did not observe a GRPC flow control seems to be normal (apart from streams showing long times to actually read data, although the amount of pending data is not large enough to warrant flow control issues).
Notice also the elevated The outbox error occurred because of a closing transport:
Which happens before the above log messages. No mention of a canceled context though. Are transport issues causing the canceled context or is a canceled context causing transport issues? Reproducing now with instrumentation in The theory that a transport issue was causing context cancellation was briefly explored, focusing on rpc heartbeat failures canceling Line 755 in 295b6ae
Although we've moved away from that as we would probably be seeing deadline exceeded errors, and I'm not sure this cancellation would propagate anywhere.
We also briefly looked at whether the TCP connection was saturated, @ajwerner suggested getting some throughput numbers and it seems that in this new run between the offending nodes, we reach
I wonder if anything weird jumps out to anyone here. One interesting thing that we noticed was a drop in heartbeat latencies for a period, going back up to previous levels at exactly the same time as the reproduction: Unfortunately the logs had rolled over (due to extra instrumentation) so we weren't able to observe anything strange in the logs. This is pretty weird, we're wondering what's going on here and how it would affect what we're seeing (cc @petermattis). In my current run, I'm observing a similar (but inverse) effect with no errors yet (RTT in the elevated stage is 370ms between concerned nodes): Hopefully will get more insight in this next reproduction as to the context cancelation but posting this in case any of this information stands out to anyone. cc @ajwerner in case you want to add anything |
So it seems that the context cancellation occurs because GRPC keepalives are timing out which results in a closed connection. Adding a stack trace when a connection closes shows the following:
The node that closes the connection doesn't seem to exhibit slow requests before this, however the receiving node seems to have been experiencing slow node liveness heartbeats (note that the node that closes the connection is the leaseholder) for 7s before that:
The fact that pings are timing out definitely rules out GRPC flow control, as these are not subject to flow control. What could slow them down on the sending side at this level seems to be possible an overloaded Here are the merged (client Something interesting I don't know what to think of is that |
What's the status here? We're seeing this in tpcc/nodes=3/w=max (on AWS). That test blocks the release.
|
FWIW I ran this manually yesterday and got a pass:
|
Nothing has been done since I last posted; this slipped in my priority queue given that there didn't seem an urgent need to fix especially with the circuit breaker fixes.
|
@jordanlewis have we looked at this recently? seems like we should make sure we can address this sooner rather than later |
I yet have to dig into the details of this, but it seems that the with the PR #38484, we have a roachtest that reproduces this readily. I'm starting to look into that now. |
I'm a little bit late to this party. I too find this to be a problem. In debugging node liveness outages I noticed that client connections from time to time just spontaneously close. I suspect that's related to grpc/grpc-go#1905 and am in the process of updating gRPC #39041. I suspect that these unexpected connection closing events exacerbated this |
It feels like we're using breakers in too many places. We baked them into the process of dialing to make sure everybody felt the effect of the breaker, but now it turns out that that's sometimes the wrong thing to do. |
tl;dr The current workaround is to
SET CLUSTER SETTING sql.distsql.flow_stream_timeout=<high value>
to increase the wait period. This results in swapping errors for higher latencies in affected queries.This issue has been observed in the wild a couple of times in 16+ node clusters with high inter-node latencies or other network slowdowns.
What happens is that the gateway sends out two or more
FlowSetupRequest
s to nodes that will be communicating with each other, but when setting up a stream, either a sending or a receiving node does not set up their flow in time (10s
default) so the query errors out withno inbound stream connection
. This is an issue because this 10s timeout should never be hit in practice.This is not due to #27746, because this seems to happen even when flows do not hit the 500 limit.
#27753 would probably fix this but it would be good to get down to the cause of this issue. i.e. why is it taking so long for a node to set up a flow?
One theory is that we only have 16 workers on the gateway node that send out
SetupFlowRequest
RPCs. Maybe those are being flooded with too many requests and some are not being sent out in time. Network slowdowns could also exacerbate this.So far it has been very difficult to reproduce this. I managed to do so with
n1-standard-16
machines running tpcc only after2h30min
by adding network delay between the nodes:Decreasing the flow timeout
And running tpcc with a
stockLevel=1
mix, to stress flow creationcc @nvanbenschoten @jordanlewis @andreimatei
The text was updated successfully, but these errors were encountered: