kvcoord: fix extremely rare OOM hazard in the DistSender#88506
kvcoord: fix extremely rare OOM hazard in the DistSender#88506craig[bot] merged 1 commit intocockroachdb:masterfrom
Conversation
9d6f2e6 to
8424cea
Compare
nvb
left a comment
There was a problem hiding this comment.
Nice catch! It looks like this is fallout from 7fb06a2#diff-271a3179bf7f12a2ed29b993611bf24f4c3c4a582a785d5d29a61d2d02c5f21fL1553, where we removed the needsTruncate argument and logic from sendPartialBatch but continued passing the untruncated rs span. So we won't need to backport this to v22.1 or earlier, but we'll want to get this back to v22.2 as soon as possible.
Reviewed 3 of 3 files at r1, all commit messages.
Reviewable status:complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)
-- commits line 17 at r1:
It's probably worth pointing at 7fb06a2 in the commit message, in case someone wants to understand this in the future.
pkg/kv/kvclient/kvcoord/dist_sender_test.go line 4604 at r1 (raw file):
splitRDB := mockRangeDescriptorDBForDescs(splitDescs...) var numAttempts atomic.Int32
atomic.Int32 is new as of Go 1.19. I thought we had kept the go directive in the go.mod file at 1.17 so that new features (i.e. generics) could not be used yet. And yet, CI is green. @rickystewart do you know what's going on here?
In the meantime, let's switch this back to a var numAttempts int32 and use the old atomic.AddInt32 and atomic.LoadInt32 so we can proceed with this PR.
pkg/kv/kvclient/kvcoord/dist_sender_test.go line 4637 at r1 (raw file):
require.True(t, testutils.IsError(pErr.GoError(), "boom")) // 6 attempts each for the two partial batches. require.Equal(t, int32(12), numAttempts.Load())
How exactly does this fail without your fix?
nvb
left a comment
There was a problem hiding this comment.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @arulajmani)
pkg/kv/kvclient/kvcoord/dist_sender.go line 1535 at r1 (raw file):
} // sendPartialBatch sends the supplied batch to the range specified by desc.
s/by desc/by the routing token/
pkg/kv/kvclient/kvcoord/dist_sender.go line 1539 at r1 (raw file):
// The batch request is supposed to be truncated already so that it contains // only requests which intersect the range descriptor and keys for each request // are limited to the range's key span. positions argument describes how the
Consider updating this comment to mention the relationship between rs and the current range descriptor (in routingTok).
|
Great find! Just for my understanding, in #87167 did we actually hit the "extremely rare OOM hazard"? We do seem to get a very large stack which points to yes, but then also there wasn't an extremely deep DistSender stack there. |
|
Yeah, great find @arulajmani, thanks for tracking this down!
I'm pretty sure that DistSender goroutine was to blame for the stack overflow, but I agree that the stack was suspiciously shallow for a 1 GB stack. I can only assume that the stack frames were very large or some such. Maybe this is better in Go 1.19 given the new stack handling? Might be worth having a look at some typical stack sizes in steady-state clusters and see what's up, the runtime has some metrics for it. |
8424cea to
6afeb65
Compare
The outer loop in the DistSender subdivides a BatchRequest into partial batches corresponding to a single range and sends them out. Previously, the resolved span passed into the function responsible for sending out this partial batch (`sendPartialBatch`) corresponded to the span of the entire batch request (as opposed to the partial batch). This resolved span is used to check if the request needs to be subdivided further between retries in the outer loop of the DistSender. Given the wrong parameter value, we'd always end up determining that the batch needed to be subdivided. This wasn't really an issue in practice as we don't expect too many retries here. However, if we did (eg. timeouts causing the transport to be exhausted on every try), the infinite recursion here could lead to an OOM. The issue here was introduced in #7fb06a22d6b5ac19f764306bdb43133946da9664 when we stopped truncating the supplied `rs` to the supplied range descriptor. References cockroachdb#87167 Release note: None
6afeb65 to
d46a1f5
Compare
arulajmani
left a comment
There was a problem hiding this comment.
TFTR!
Might be worth having a look at some typical stack sizes in steady-state clusters
I looked at the DistSender stack, and doing some stack pointer/frame pointer math, these stack frames seem to be around ~2KB. The stack doesn't seem deep enough to hit that 1GB limit, so maybe there's more to look at here. I'll reword the commit message so that merging this PR doesn't close the issue.
Reviewable status:
complete! 0 of 0 LGTMs obtained (waiting on @nvanbenschoten and @rickystewart)
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
It's probably worth pointing at 7fb06a2 in the commit message, in case someone wants to understand this in the future.
Done.
pkg/kv/kvclient/kvcoord/dist_sender.go line 1539 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
Consider updating this comment to mention the relationship between
rsand the current range descriptor (inroutingTok).
Added more words here.
pkg/kv/kvclient/kvcoord/dist_sender_test.go line 4604 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
atomic.Int32is new as of Go 1.19. I thought we had kept thegodirective in thego.modfile at 1.17 so that new features (i.e. generics) could not be used yet. And yet, CI is green. @rickystewart do you know what's going on here?In the meantime, let's switch this back to a
var numAttempts int32and use the oldatomic.AddInt32andatomic.LoadInt32so we can proceed with this PR.
I didn't realize this was a new feature. Great advert for GoLand autocomplete though haha
pkg/kv/kvclient/kvcoord/dist_sender_test.go line 4637 at r1 (raw file):
Previously, nvanbenschoten (Nathan VanBenschoten) wrote…
How exactly does this fail without your fix?
It would just timeout because the maxRetries we set above won't be respected. It's not the greatest test, given we don't configure this in production; maybe we should?
|
bors r=nvanbenschoten |
|
Build failed (retrying...): |
|
Build succeeded: |
I also don't see how this adds up to 1GB. Subtracting the first stack pointer from the last in this goroutine does not give 1GB. And yet, we see a matching stack pointer in the panic text: Notice that So this goroutine does seem to be at fault. |
The outer loop in the DistSender subdivides a BathRequest into partial batches corresponding to a single range and sends them out. Previously, the resolved span passed into the function responsible for sending out this partial batch (
sendPartialBatch) corresponded to the span of the entire batch request (as opposed to the partial batch). This resolved span is used to check if the request needs to be subdivided further between retries in the outer loop of the DistSender. Given the wrong parameter value, we'd always end up determining that the batch needed to be subdivided.This wasn't really an issue in practice as we don't expect too many retries here. However, if we did (eg. timeouts causing the transport to be exhausted on every try), the infinite recursion here could lead to an OOM.
References #87167
Release note: None