-
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
stability: investigate indigo issues #13687
Comments
So indigo was running with a .default set to:
I've removed this, but I keep seeing the same issue in the logs: Which is currently being tracked in #13042 |
I wiped and restarted This corresponded with 1 range being reported as under-replicated: I stopped |
Started the Seems like there is a stuck snapshot operation: Looking at the running goroutines I see:
The above is on indigo 8. On the sender side (indigo 2) the logs show:
I'm not sure what to make of all this yet. |
I restarted indigo 8. After restarting, another snapshot wedged up almost immediately. On the sender side (indigo 2) I see:
So indigo 2 tried to send a snapshot for Anyone have any thoughts on why we can't send the snapshot in 1m? I'm going to try and add some more log messages to pinpoint what is getting horked up here. |
Also, why does indigo 8 seem to never get notification that the snapshot operation timed out? @tamird Is there something funky going on with context cancellation here? |
I wouldn't normally expect this to have anything to do with context cancellation; indigo 8 should be noticing if the sender goes away when it calls Recv or Send on the stream. That said, if the sender has a timeout (why does it have one, btw?), maybe we should be watching the stream context (which will have a propagated timeout) and bail from the RPC method. |
The sender has a 1m timeout because it was started from the Raft snapshot queue. Queues have a default 1m timeout. The recipient is stuck on |
diff --git a/pkg/storage/raft_transport.go b/pkg/storage/raft_transport.go
index e228fb0a0..f51bca5bf 100644
--- a/pkg/storage/raft_transport.go
+++ b/pkg/storage/raft_transport.go
@@ -337,8 +337,9 @@ func (t *RaftTransport) RaftMessageBatch(stream MultiRaft_RaftMessageBatchServer
// RaftSnapshot handles incoming streaming snapshot requests.
func (t *RaftTransport) RaftSnapshot(stream MultiRaft_RaftSnapshotServer) error {
+ ctx := stream.Context()
errCh := make(chan error, 1)
- if err := t.rpcContext.Stopper.RunAsyncTask(stream.Context(), func(ctx context.Context) {
+ if err := t.rpcContext.Stopper.RunAsyncTask(ctx, func(ctx context.Context) {
errCh <- func() error {
req, err := stream.Recv()
if err != nil {
@@ -365,6 +366,8 @@ func (t *RaftTransport) RaftSnapshot(stream MultiRaft_RaftSnapshotServer) error
return err
}
select {
+ case <-ctx.Done():
+ return ctx.Err()
case <-t.rpcContext.Stopper.ShouldStop():
return nil
case err := <-errCh:
Breaking out of the function should unwedge the |
Oh, yeah, that's not good. That explains why the snapshot is getting wedged on the recipient. And I think it is timing out because we're only able to send ~1MB/sec and the range is larger than 60MB. |
Hmm, a simple bandwidth test using |
Where did the 1 MB/s figure come from?
…On Feb 24, 2017 14:27, "Peter Mattis" ***@***.***> wrote:
Hmm, a simple bandwidth test using nc shows we can send 40MB/sec between
indigo 2 and indigo 8.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#13687 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/ABdsPL2mONHleG6HE0RdWGkB2__UA2mPks5rfy8sgaJpZM4MHXZP>
.
|
Log messages in |
I'm suspicious that indigo 8 is falling behind simply because it has higher latency. Ping times between indigo 2 and indigo 5 (the other node with a replica) are 40ms. Between indigo 2 and indigo 8 the ping times are 60ms. The Raft log is likely growing to a size where we're truncating it and forcing a snapshot. But there really isn't another choice here. Seems like we need to have flow control ala #8659. |
Bump the snapshot operation timeout from 1m to 5m. We've seen in production that sending O(64MB) can take longer than 1m on a cross-region link. (Need to investigate why this is happening, though). Properly notice cancellation of snapshot streams on the recipient. Failing to do this would cause a reception of a snapshot to wedge permanently if the associated operation timed out. See cockroachdb#13687
We do ultimately need flow control, but in this case we should really be able to send the snapshot in under a minute. I suspect that we're hitting some sort of pathological case in grpc or its http/2 layer that might explain both this and #13722. |
Even if we can send snapshots faster, the fact that one Replica is "slower" than the other 2 means it will fall farther and farther behind until a snapshot is needed. Hence the need for flow control. |
A higher-latency replica is not automatically doomed to fall behind; it should be able to keep up as long as it has sufficient bandwidth (of course the two tend to be correlated) |
Regardless of the reason a replica is continually falling behind (latency, bandwidth, slowness on the node itself), without flow control we'll get into these snapshot loops. But before flow control, I'm going to do a bit of experimentation to try and figure out why our bandwidth via gRPC seems dramatically lower than what |
I wrote a simple gRPC program that implements the following service:
And used some of the looping logic from
And on indigo 2:
Wtf? For comparison, an
And the
But even this is sad as |
gRPC has a default "window size" of 64KB. This is the maximum amount of data that can be sent on a stream before blocking until the data is acknowledged. With a 60ms RTT, sending 64KB at a time results in a bandwidth of |
I've been looking at GRPC's HTTP/2 window logic. It does not attempt to grow the window; it just uses a fixed 64KB per stream and 1024KB per connection. Looks like we need to either make it configurable or make it grow dynamically. |
Yeah, that's what I'm seeing too. I also found grpc/grpc-go#760. Manually bumping the window size by 10x allows my |
Java gRPC uses a default flow control window size of 1MB (16x the Go gRPC default): https://github.com/grpc/grpc-java/blob/master/netty/src/main/java/io/grpc/netty/NettyChannelBuilder.java#L72 |
Does pinger do better than 7MB/s with 16x?
…On Fri, Feb 24, 2017 at 6:58 PM Peter Mattis ***@***.***> wrote:
Java gRPC uses a default flow control window size of 1MB (16x the Go gRPC
default):
https://github.com/grpc/grpc-java/blob/master/netty/src/main/java/io/grpc/netty/NettyChannelBuilder.java#L72
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#13687 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AF3MTd3a8_sNGN8w_a3pca9Aa3hOxN0Wks5rf5jGgaJpZM4MHXZP>
.
|
Yes, but you have to set the size of each RPC larger than 1MB. Even doing that, I've only seen the bandwidth hit 10MB/s. |
There are two window size settings, one for the stream and one for the connection. Are you increasing both? (The linked java code is used for the connection window. I can't find whether the same value is used for the connection window). |
While this is being fixed upstream, I wonder if we should have a local gRPC fork where we bump Also, we don't have compression enabled. At the time we switch to gRPC, I don't think there were hooks for compression, but now there are. |
The region on the left is with the default window size settings. On the right is a manual edit to grpc where I bump |
I'm happy to look at flow control next.
…On Mon, Feb 27, 2017 at 9:16 AM Peter Mattis ***@***.***> wrote:
[image: screen shot 2017-02-27 at 12 14 03 pm]
<https://cloud.githubusercontent.com/assets/6201335/23371500/48db5772-fce6-11e6-824b-54a2282a3fb9.png>
The region on the left is with the default window size settings. On the
right is a manual edit to grpc where I bump initialWindowSize and
initialConnWindowSize to 2MB each. Much improved, but note that this is
only improving the speed of snapshots. We're still generating snapshots
frequently because of Raft log truncation and we're truncating the Raft log
and forcing Raft snapshots because 1 of the replicas cannot keep up.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#13687 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AF3MTbp2G-XVtIW0wmdsEVRaEnRh8C5cks5rgwTdgaJpZM4MHXZP>
.
|
I'm already looking at it. |
I wonder if our default setting for |
Yeah, the combination of |
I'm testing various values of |
The above shows 16 experiments varying max-inflight-msgs from 4, 16, 64 and 256 and raft log max-size from 1MB, 2MB, 4MB and 8MB. The short story is that setting max-inflight-msgs to 4 is too low as 3 out of the 4 runs with that setting show problematic Raft log growth. Curiously, setting max-inflight-msgs to 256 also seems to be mildly problematic and I have no explanation for that. Given this experiment, I'm going to default max-inflight-msgs to 64 and Raft log max-size to 4MB. We'll want to pay attention to what this does to chaos testing on |
Increase Raft MaxInFlightMsgs default from 4 to 64 which allows deeper pipeling of Raft messages on high latency links. Increase Raft log max-size from 1MB to 4MB. The 1MB setting was overly aggressive on high latency links and seems to contribute to increased likelihood of Raft snapshots. See cockroachdb#13687
That is interesting. I assume the experiments ran as inflight=4,
max-size=1MB, inflight=16, max-size=1MB, inflight=64, max-size=1MB, etc...?
…On Tue, Feb 28, 2017 at 6:54 PM Peter Mattis ***@***.***> wrote:
[image: screen shot 2017-02-28 at 6 47 58 pm]
<https://cloud.githubusercontent.com/assets/6201335/23439814/83f0251c-fde6-11e6-8111-bc1cd335672a.png>
The above shows 16 experiments varying max-inflight-msgs from 4, 16, 64
and 256 and raft log max-size from 1MB, 2MB, 4MB and 8MB. The short story
is that setting max-inflight-msgs to 4 is too low as 3 out of the 4 runs
with that setting show problematic Raft log growth. Curiously, setting
max-inflight-msgs to 256 also seems to be mildly problematic and I have no
explanation for that.
Given this experiment, I'm going to default max-inflight-msgs to 64 and
Raft log max-size to 4MB. We'll want to pay attention to what this does to
chaos testing on blue.
—
You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
<#13687 (comment)>,
or mute the thread
<https://github.com/notifications/unsubscribe-auth/AF3MTdVxCKSHzFnG-BxAJEpHY_Rl7Eraks5rhLOvgaJpZM4MHXZP>
.
|
Yes, exactly. |
I think this is resolved. Please re-open if not. |
Indigo keeps breaking, throwing everything from zero-qps, liveness != nodes_ups, and leader-not-leaseholder alerts.
Alerts are currently silenced.
The text was updated successfully, but these errors were encountered: