-
Notifications
You must be signed in to change notification settings - Fork 107
Limit series from clusterByFind operation #1021
Conversation
|
||
// 0 disables the check, so only check if maxSeriesPerReq > 0 | ||
if maxSeriesPerReq > 0 && len(resp.Metrics)+len(allSeries) > maxSeries { | ||
return nil, |
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.
Shouldn't you cancel ctx
so outstanding peer queries are aborted?
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.
Technically, the query will be Done
d when the error propagates up and a response is returned. But canceling it here could do it a little bit earlier.
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.
I'm also not entirely sure how to cancel this context here. I suppose we would need to make a new cancelable context here.
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.
that makes sense I think. Since we only get the ctx passed in, we don't know where what the cancelfunc was.
However, if you do something like newCtx, cancel := context.WithCancel(ctx)
and pass newCtx into peerQuerySpeculativeChan, then you can cancel it.
I plan on following up this change with a version that decodes directly into a |
See master...bloomberg:directDecode for a version of this change that includes local request processing and decoding straight from the |
@@ -294,7 +294,7 @@ func (s *Server) peerQuery(ctx context.Context, data cluster.Traceable, name, pa | |||
result := make(map[string]PeerResponse) | |||
for resp := range responses { | |||
if resp.err != nil { | |||
return nil, err | |||
return nil, resp.err |
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.
oops!
log.Debug("HTTP Render querying %s%s", peer.GetName(), path) | ||
buf, err := peer.Post(reqCtx, name, path, data) | ||
// peerQuerySpeculativeChan takes a request and the path to request it on, then fans it out | ||
// across the cluster. If any peer fails requests to other peers are aborted. If enough |
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.
I notice peerQuerySpeculative
description includes "...except to the local peer...", whereas this one does not.
i think the former is wrong
api/graphite.go
Outdated
// 0 disables the check, so only check if maxSeriesPerReq > 0 | ||
if maxSeriesPerReq > 0 && len(resp.Metrics)+len(allSeries) > maxSeries { | ||
return nil, | ||
response.NewError(413, fmt.Sprintf("Request exceeds max-series-per-req limit (%d). Reduce the number of targets or ask your admin to increase the limit.", maxSeriesPerReq)) |
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.
please use the http.StatusXxx constants
in this case http.StatusRequestEntityTooLarge
} | ||
go func() { | ||
defer close(errorChan) | ||
defer close(resultChan) |
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.
minor thought: would it be cleaner/simpler to just have 1 return channel? we could return PeerResponse along with its error
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.
Possibly. I like the 2 channels, because you can just loop over responses until it is closed, then quickly try to read into an err var for the return value. See this usage
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.
fair enough
sounds great. but I suggest we first merge this, and then new PR for that work. Do you have interesting observations/stats you can share with us? |
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.
looks pretty good. few minor tweaks needed
I definitely saw a reduction in short-term heap spikes, but overall heap usage stayed about the same. We've been running this in prod for about 2 months now, without any issues. |
|
||
var allSeries []Series | ||
|
||
for r := range responseChan { | ||
resp := models.IndexFindByTagResp{} | ||
_, err := resp.UnmarshalMsg(r.buf) | ||
if err != nil { | ||
cancel() |
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.
Would it make sense to just defer cancel()
at the top like the peerQuery functions do?
Also, how much does this canceling really buy us if the top-level ctx will be Done
when the error is returned?
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.
not sure what you mean with a ctx "being Done".
ctx.Done()
just returns a channel that signals cancellation (by getting closed), but you still need to call a corresponding cancel function for the cancellation to happen.
I see 3 callers of this function:
/render
->renderMetrics
->executePlan
/tags/findSeries
(graphiteTagFindSeries)querier.Select
we would need all 3 to call a cancelfunc for my change to be unnecessary.
but from what i can tell, it looks like neither of them, or at least not the first two, do this. (unless macron automatically calls cancel on our behalf?)
but regardless, it shouldn't take much code spelunking for reader to assert whether we cancel a context when we should. so i rather call cancel wherever it seems sensible, even if we call it multiple times (which is harmless).
I think @woodsaj was trying to establish this convention when he first added cancellations throughout this code base.
as for using defer vs just putting the code at the return sites, for short functions it's really debatable and depends more on taste. so unless you have a strong argument, let's just stick with it.
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.
Re: Done
I mean that peerQuerySpeculativelyChan
checks if the context is Done()
https://github.com/grafana/metrictank/pull/1021/files#diff-bc8a656be21edce0cf2a74adf23d7aeaR401
Done isn't just done on cancel, but also on a final response being returned. So, as soon as the error propagates up and a response is delivered, peerQuerySpeculativelyChan
will break its loop and call cancel()
. This is done specifically so callers don't need to implement cancellation themselves. Granted, doing it directly will cancel a few milliseconds earlier, so might be marginally cheaper.
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.
the cancels that i added in clusterFindByTag target cases that peerQuerySpeculativeChan
cannot detect itself. these are :
- unmarshalling error
- exceeding maxSeriesPerReq
the cancels triggered in peerQuerySpeculativeChan
(upon erroring peer.Post and upon function return) would be triggered not all, or much later, respectively.
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.
the cancels that i added in clusterFindByTag target cases that peerQuerySpeculativeChan cannot detect itself.
True, but as long as these failures cause an error response to the HTTP request, the parent context's Done
channel will be closed and peerQuerySpeculativeChan
will short-circuit and will call cancel
on the context it created.
This is the case in the 3 examples you posted, but I guess it's too much for clusterFindByTag
to assume?
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.
the parent context's Done channel will be closed
where does this happen?
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.
TBH, I didn't come to this conclusion by code inspection, but rather by logging when early responses happened. It definitely was how it behaved, but I couldn't tell you if it's by contract or a symptom of how I was querying
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.
I see, well. this reinforces my point that these matters should be made much more obvious. hence rather a call to cancel more rather than too few.
good to merge?
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.
Yeah, let me rebase
80afcab
to
fa4cc1e
Compare
LGTM. I see what looks like an unrelated failure in the tests. |
yes, this failure is due to the flakey TimeLimiter tests. see above PR. |
nice work sean! |
Fixes #968 and #1018
Looking through the node::Post code, it seems like we could save quite a lot by marshaling directly from the response body into the msgp generated type. My C++ brain wants to use templates to accomplish it, so I'm not sure the cleanest method to use in go without generics. For that reason, I punted on that for now.