> On May 13, 2015, 11:50 p.m., Aditya Auradkar wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java,
> >  line 45
> > <https://reviews.apache.org/r/34170/diff/2/?file=958605#file958605line45>
> >
> >     I think this is a good catch.
> >     
> >     Just so I understand, this is adding new samples for blank intervals of 
> > time when record was not called. As an optimization, I think you should 
> > only add "MetricConfig.samples" number of samples. In the rare case, there 
> > is no activity for 10 mins (say), this will add 10sec*6*10 = 600 samples 
> > which will be purged immediately on the next record call.
> 
> Jay Kreps wrote:
>     This is a really good point. It is totally possible for a metric to track 
> activity on a topic that has no writes for a month, the first write would 
> then cause you to cycle through a month of samples. The logic around 
> correctly skipping these windows and calculating the correct window boundry 
> needs to be carefully worked out.
> 
> Jay Kreps wrote:
>     Actually maybe you can elaborate on why this is needed at all? In the 
> current code if the current sample is complete we add a new sample. Your code 
> adds lots of samples. But why do we need that? Isn't purging obsolete samples 
> handled on measurement already? I think that is more elegant. You must see 
> some issue there, maybe you can explain?
> 
> Dong Lin wrote:
>     Adi: Good catch! I will update the patch to fix the problem.
>     
>     Jay: Sure. Here is the problem I find with the current code.
>     
>     - The current code implements SampledStat.advance in a way that is 
> probably different from users' expectation: typically when rate is sampled, 
> time will be divided into consecutive slots of equal length and samples are 
> expired in unit of these time slots, which assures the user the rate is 
> measured on samples collected in the past expireAge period. The current 
> implementation doesn't provide such an easy-to-understand interpretations, 
> since the effective sampled period can be anywhere between 0 and expireAge.
>     
>     - As one extreme example, say we call rate.measure after expireAge (i.e. 
> config.samples() * config.timeWindowMs()) has passed since last sample. Then 
> purgeObsoleteSamples will reset all samples and elapsed will be 0 as a 
> result. This is problematic: rate should instead keep the information that we 
> haven’t observed any sample during the past expireAge.
>     
>     Does it make sense?
> 
> Jay Kreps wrote:
>     Hey Dong, I think what you are saying is that the last sample is partial, 
> that is it doesn't cover a full window yet. This is true and is by design. 
> That is the whole reason there are multiple samples--to stabalize the partial 
> estimate. The ideal window would be a backwards looking sliding window of N 
> ms from the time of measurement but that would be computationally unfeasible.
>     
>     I don't understand your example. If no events have occurred in 
> config.samples() * config.timeWindowMs() then the observed rate is 0, right? 
> I think you are saying that the time estimate should be the full window, but 
> that is fixed by changing the other formula as I suggested.
> 
> Dong Lin wrote:
>     Hey Jay, thanks for reply. The problem I wanted to explain with this 
> example is that, if no events have occurred in config.samples() * 
> config.timeWindowMs(), oldest == now and the measured rate will be infinite 
> (which should be 0 instead).
>     
>     The example explained above is only relevant to the current code. As you 
> said, using the other formula you suggested will fix the problem and avoid 
> the need to make this change here.
>     
>     Just a couple of questions towards an acceptable patch. How about we use 
> the other formula you suggested in Rate.measure()? When the user calls 
> rate.measure() with number of samples <= 1 (i.e. ellapsed <= timeWindowMs), 
> should we throw exception or return n/timeWindowMs? And should we keep the 
> existing SampledStat.oldest()?
>     
>     Thank much,
> 
> Jay Kreps wrote:
>     Ah gotcha, yeah i think it is easiest to fix in measure() rather than in 
> advance.
>     
>     You definitely don't want to throw an exception in measure because it is 
> totally correct to have any number of samples when measure is called. I think 
> the proposed change was require the configured num samples (i.e. the max 
> samples) to be at least 2 and then always use the formula we discussed in 
> measure.
>     
>     But heres the thing I think this logic is simply very precise and has to 
> be carefully thought out with each change. So don't listen to me too much, 
> work it through and see if you agree that that makes sense and if you don't 
> let's talk about alternatives.

Thanks Jay for all your helpful comments! I will try to come up with the best 
approach. I will submit the patch after I test it with Adi's quota 
implementation (as part of KAFKA-2083 System Test for Quotas).


- Dong


-----------------------------------------------------------
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/34170/#review83687
-----------------------------------------------------------


On May 14, 2015, 7:34 a.m., Dong Lin wrote:
> 
> -----------------------------------------------------------
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> -----------------------------------------------------------
> 
> (Updated May 14, 2015, 7:34 a.m.)
> 
> 
> Review request for kafka.
> 
> 
> Bugs: KAFKA-2191
>     https://issues.apache.org/jira/browse/KAFKA-2191
> 
> 
> Repository: kafka
> 
> 
> Description
> -------
> 
> KAFKA-2191; Measured rate should not be infinite
> 
> 
> Diffs
> -----
> 
>   clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java 
> 98429da34418f7f1deba1b5e44e2e6025212edb3 
>   
> clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java 
> b341b7daaa10204906d78b812fb05fd27bc69373 
> 
> Diff: https://reviews.apache.org/r/34170/diff/
> 
> 
> Testing
> -------
> 
> 
> Thanks,
> 
> Dong Lin
> 
>

Reply via email to