-
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
rpc: enable dynamic window resizing #35161
Conversation
This commit removes the custom GRPC stream and connection window size added due to #13687. Since then, GRPC has added dynamic window resizing, resulting in window resizing depending on the bandwidth-delay product of a stream (i.e. increase the window if data is flowing). However, GRPC only enables this behavior if the initial window size is less than or equal to the default window size. The reason to have large initial window sizes was to allow for reasonable throughput on high-latency links. However, the trouble with having a static window is that some streams can block other streams by taking up the total window size even though nobody is reading from it on the other side. Dynamic window sizing improves on both cases. Firstly, since the initial size is smaller than our current default, any blocked streams will only occupy 64KB (the default window size) instead of 2MB (our custom window size). Secondly, if high throughput is needed (e.g. for a snapshot), we can use up to 4MB of window space on the connection, resulting in double the throughput when necessary (experiments shared in this PR). Closes #14948 Noting this as a bug fix in the release note although this should also be a performance improvement for snapshots. Release note (bug fix): Avoid possible query deadlocks
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.
Nice analysis and graphs!
I'm refreshing my memory on this gRPC window size stuff. There is a per-connection window, and then per-stream windows. When we were using a fixed window size, both the per-connection and per-stream windows were set to 2MB. With the dynamic window, the per-connection window dynamically adjusts (up to 4MB) and the per-stream window starts at 64KB and will increase as the stream gets used. Is this correct?
Something else that would be worth testing is how long it takes to send 64MB in 256KB chunks (simulating sending a snapshot) with dynamic-window enabled vs disabled over various RTT links. That would alleviate your concern with the ramp up period for 120ms RTT. It would also be interesting to test sending various sizes of data (2MB, 4MB, 8MB, 16MB, 32MB) in a single chunk as that would simulate a large unary RPC such as a table scan. I'm hoping this will be easy to do with your existing setup.
Can you expand the Release note
text a bit and explain why this change avoids possible query deadlocks?
Reviewable status: complete! 1 of 0 LGTMs obtained (waiting on @petermattis)
My assumption that new streams on an existing connection start with the default window size is wrong, so this patch won't help with regards to deadlocking when a dependent stream is blocking its dependency (e.g. left side of a hash join). BDP estimation is done at the connection level, so as long as historically things have flowed well on a connection, a new stream will have the potential to block other streams:
Quick note: the GRPC version that cockroach uses has a 4MiB max connection window size which I assumed was a limit we hit in these tests but I didn't notice that in the GRPC version that I'm using, the max connection window size has been increased to 16MiB (https://github.com/grpc/grpc-go/pull/2455/files), related to this issue (grpc/grpc-go#2400). However, on roachprod, we are effectively limited to a 6MiB max connection window:
We started off by using 2MiB for both the stream and connection window sizes but then moved to 32MiB for the connection to make it less likely that a stream could block the connection window. However, at 32MiB the connection window size is limited by TCP (as linked above). Thanks a lot for the measurement suggestions, they were very helpfull: |
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.
Thanks for the analysis, @asubiotto. Agreed that we should probably put this on ice for a while as it almost certainly doesn't fix #14948. Per offline discussion, I think you should attempt to reproduce that hung query scenario in a test setup.
Reviewable status: complete! 1 of 0 LGTMs obtained
Note: I tried to reproduce the hung query scenario and ran into grpc/grpc-go#2792. I think I might open a feature request for per-stream window resizing. Going to close this PR in the meantime. |
This commit removes the custom GRPC stream and connection window size
added due to #13687. Since then, GRPC has added dynamic window resizing,
resulting in window resizing depending on the bandwidth-delay product of
a stream (i.e. increase the window if data is flowing). However, GRPC
only enables this behavior if the initial window size is less than or
equal to the default window size.
The reason to have large initial window sizes was to allow for
reasonable throughput on high-latency links. However, the trouble with
having a static window is that some streams can block other streams by
taking up the total window size even though nobody is reading from it on
the other side. Dynamic window sizing improves on both cases. Firstly,
since the initial size is smaller than our current default, any blocked
streams will only occupy 64KB (the default window size) instead of 2MB
(our custom window size).
Secondly, if high throughput is needed (e.g. for a snapshot), we can use
up to 4MB of window space on the connection, resulting in double the
throughput when necessary (experiments shared in this PR).
Closes #14948
Noting this as a bug fix in the release note although this should also
be a performance improvement for snapshots.
Release note (bug fix): Avoid possible query deadlocks
Testing this in various RTT scenarios showed that throughput in high-latency scenarios seems to double, which is expected due to the fact that the maximum window size when using dynamic window resizing is 4MB, twice our current 2MB setting. Here is the graph (DW=F means dynamic window disabled, DW=T means dynamic window enabled)
Link to the spreadsheet: https://docs.google.com/spreadsheets/d/1M1SfFE3ltCXln-gN0vGCNhbW6N46XWaX3MqNagkf9Vk/edit#gid=0
Go program used was a modification of gupload to use cockroach's GRPC settings as closely as possible: https://github.com/asubiotto/gupload
Latency added using:
Program run using what I gathered was the maximum snapshot chunk size:
The only concern here might be that the ramp up period slightly affects us in the 120ms case (we are slightly under the static window numbers for the first 2 seconds), although I struggle to see how this will be a problem in practice. This might benefit from checking out on even higher latency links.