Re: Review Request 34170: Patch for KAFKA-2191

2015-05-27 Thread Dong Lin

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

(Updated May 28, 2015, 12:31 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 

Diff: https://reviews.apache.org/r/34170/diff/


Testing (updated)
---

For your reference, this is the patch I used in the quota system test. It 
solves the problem.

This patch follows the suggestion from Jay. Jay has a more complete patch that 
includes the testcase. Feel free to use the patch by Jay at 
https://reviews.apache.org/r/34418/diff/.


Thanks,

Dong Lin



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-27 Thread Dong Lin

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

(Updated May 28, 2015, 12:28 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 (updated)
-

  clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java 
98429da34418f7f1deba1b5e44e2e6025212edb3 

Diff: https://reviews.apache.org/r/34170/diff/


Testing
---


Thanks,

Dong Lin



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-19 Thread Dong Lin


> On May 19, 2015, 5:14 p.m., Jay Kreps wrote:
> >

Oh sure. Let me look through your patch.

Thank you.


- Dong


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


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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-19 Thread Dong Lin


> On May 19, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java,
> >  line 46
> > 
> >
> > Not sure if I grok why we need this while loop. Take a look at the 
> > patch I posted and see if you think that is correct. I added a test for the 
> > 0 ellapsed time case too.

Hey Jay,

The latest patch is updated 5 days ago, even earlier than our last discussion. 
I don't plan to use while loop as we discussed. I will post the patch after I 
complete the system test for quota implementation.


- Dong


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


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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-19 Thread Jay Kreps

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



clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java


Not sure if I grok why we need this while loop. Take a look at the patch I 
posted and see if you think that is correct. I added a test for the 0 ellapsed 
time case too.


- Jay Kreps


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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-17 Thread Dong Lin


> 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
> > 
> >
> > 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 KA

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-15 Thread Jay Kreps


> 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
> > 
> >
> > 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,

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.


- Jay


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


On M

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-15 Thread Dong Lin


> 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
> > 
> >
> > 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.

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,


- 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

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-15 Thread Jay Kreps


> 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
> > 
> >
> > 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?

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.


- Jay


---
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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-14 Thread Aditya Auradkar


> On May 13, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 
> > 62
> > 
> >
> > Is this actually right? I agree you'll get discontinuities as the 
> > measured time shrinks to zero but why is giving back 0 the right answer? In 
> > the case you guys were testing 0 was safe, but imagine a case where the 
> > monitoring was checking that the value didn't fall below some threshold.
> 
> Dong Lin wrote:
> The question is, when Rate.measure() is called right after 
> Rate.record(n), what should be the return value? I think there are two 
> possibilities: 0 and n/config.timeWindowMs(). I didn't find any use case 
> where these two values make a difference. 
> 
> Which value do you think is the best?
> 
> Thank you.
> 
> Aditya Auradkar wrote:
> I think returning 0 is reasonable if no time has elapsed (technically).
> As an alternate solution, what if we assumed that "elapsed" is always 1 
> (at least). For example:
> 
> double elapsed = convert(now - stat.oldest(now).lastWindowMs) + 1
> 
> In case of seconds, this basically means that you assume the current 
> second is always complete. This is only a problem (for a couple of seconds) 
> when all previous samples have zero activity or when the server is just 
> starting up.
> 
> Jay Kreps wrote:
> Actually there is a fundamental issue with the computation that patching 
> around the 0 case doesn't fix. That is the instability of the estimate. This 
> is the "taking a poll with sample size one" problem.
> 
> Even if you patch the 0 case you still get a bad answer 1 ms later. That 
> is, let's say you get a single 50k request and your quota is 1MB/sec. 
> Currently at 0ms we estimate infinity which is in fact the measured rate but 
> obviously not a good estimate. But even 1 ms later the estimate is bad. 
> 50k*1000ms = ~50MB/sec.
> 
> This is somewhat rare because it only happens when there is just one 
> sample.
> 
> They key observation is that if a sample is missing, nothing happened in 
> that time period. But the calculation should still use that time period.
> 
> So the right way to compute it, I think, is actually
>   ellapsed = (num_samples-1)*window_size + (now - current_sample.begin)
>   
> For safety I think we should also require the number of samples to be >= 
> 2 and default it to 3.
> 
> Dong Lin wrote:
> Hi Jay: thanks for comments!
> 
> I think the problem here is that, when event number is very small (e.g. 
> 0, 1), what value should rate.measure() return, right? If I understand your 
> solution formula right, when there is only one sample, your measured rate is 
> n/ellapsed = n/(now - current_sample.begin), which is exactly same as the 
> current code. I agree that requiring number of samples to be >= 2 solve the 
> problem. But what happens when user call rate.measure() when there is only 1 
> sample?
> 
> I agree with you that we still get a bad answer if we patch the 0 case. 
> How about we patch the 0 to timeWindowMs case: if (ellapsed < timeWindowMs) 
> then ellapsed = timeWindowMs. Does this solve the problem here?
> 
> Dong Lin wrote:
> I think we all agree that there is problem with rate measure when 
> ellapsed time is too small. What we need to properly define the rate measure 
> to handle such case, which is currently missing.
> 
> If we want to require number of samples to be >= 2 as Jay suggested, we 
> can also let ellapsed = max(ellapsed, 2*timeWindowMs) and keep the rest of 
> the code, right?

If we require num samples greater than 2, we shoud simply change this condition 
in MetricConfig:
if (samples < 1)
throw new IllegalArgumentException("The number of samples must be 
at least 1.");

Jay's equation is now similar to Dongs.

If num samples = 3,
ellapsed = (num_samples-1)*window_size + (now - current_sample.begin)
ellapsed = 2*window_size + (now - current_sample.begin)


- Aditya


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


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 
> 98429da3

Re: Review Request 34170: Patch for KAFKA-2191

2015-05-14 Thread Dong Lin

---
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 (updated)
-

  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



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-14 Thread Dong Lin


> On May 13, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 
> > 62
> > 
> >
> > Is this actually right? I agree you'll get discontinuities as the 
> > measured time shrinks to zero but why is giving back 0 the right answer? In 
> > the case you guys were testing 0 was safe, but imagine a case where the 
> > monitoring was checking that the value didn't fall below some threshold.
> 
> Dong Lin wrote:
> The question is, when Rate.measure() is called right after 
> Rate.record(n), what should be the return value? I think there are two 
> possibilities: 0 and n/config.timeWindowMs(). I didn't find any use case 
> where these two values make a difference. 
> 
> Which value do you think is the best?
> 
> Thank you.
> 
> Aditya Auradkar wrote:
> I think returning 0 is reasonable if no time has elapsed (technically).
> As an alternate solution, what if we assumed that "elapsed" is always 1 
> (at least). For example:
> 
> double elapsed = convert(now - stat.oldest(now).lastWindowMs) + 1
> 
> In case of seconds, this basically means that you assume the current 
> second is always complete. This is only a problem (for a couple of seconds) 
> when all previous samples have zero activity or when the server is just 
> starting up.
> 
> Jay Kreps wrote:
> Actually there is a fundamental issue with the computation that patching 
> around the 0 case doesn't fix. That is the instability of the estimate. This 
> is the "taking a poll with sample size one" problem.
> 
> Even if you patch the 0 case you still get a bad answer 1 ms later. That 
> is, let's say you get a single 50k request and your quota is 1MB/sec. 
> Currently at 0ms we estimate infinity which is in fact the measured rate but 
> obviously not a good estimate. But even 1 ms later the estimate is bad. 
> 50k*1000ms = ~50MB/sec.
> 
> This is somewhat rare because it only happens when there is just one 
> sample.
> 
> They key observation is that if a sample is missing, nothing happened in 
> that time period. But the calculation should still use that time period.
> 
> So the right way to compute it, I think, is actually
>   ellapsed = (num_samples-1)*window_size + (now - current_sample.begin)
>   
> For safety I think we should also require the number of samples to be >= 
> 2 and default it to 3.
> 
> Dong Lin wrote:
> Hi Jay: thanks for comments!
> 
> I think the problem here is that, when event number is very small (e.g. 
> 0, 1), what value should rate.measure() return, right? If I understand your 
> solution formula right, when there is only one sample, your measured rate is 
> n/ellapsed = n/(now - current_sample.begin), which is exactly same as the 
> current code. I agree that requiring number of samples to be >= 2 solve the 
> problem. But what happens when user call rate.measure() when there is only 1 
> sample?
> 
> I agree with you that we still get a bad answer if we patch the 0 case. 
> How about we patch the 0 to timeWindowMs case: if (ellapsed < timeWindowMs) 
> then ellapsed = timeWindowMs. Does this solve the problem here?

I think we all agree that there is problem with rate measure when ellapsed time 
is too small. What we need to properly define the rate measure to handle such 
case, which is currently missing.

If we want to require number of samples to be >= 2 as Jay suggested, we can 
also let ellapsed = max(ellapsed, 2*timeWindowMs) and keep the rest of the 
code, right?


- Dong


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


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Dong Lin


> 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
> > 
> >
> > 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?

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?


- Dong


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


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Dong Lin


> On May 13, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 
> > 62
> > 
> >
> > Is this actually right? I agree you'll get discontinuities as the 
> > measured time shrinks to zero but why is giving back 0 the right answer? In 
> > the case you guys were testing 0 was safe, but imagine a case where the 
> > monitoring was checking that the value didn't fall below some threshold.
> 
> Dong Lin wrote:
> The question is, when Rate.measure() is called right after 
> Rate.record(n), what should be the return value? I think there are two 
> possibilities: 0 and n/config.timeWindowMs(). I didn't find any use case 
> where these two values make a difference. 
> 
> Which value do you think is the best?
> 
> Thank you.
> 
> Aditya Auradkar wrote:
> I think returning 0 is reasonable if no time has elapsed (technically).
> As an alternate solution, what if we assumed that "elapsed" is always 1 
> (at least). For example:
> 
> double elapsed = convert(now - stat.oldest(now).lastWindowMs) + 1
> 
> In case of seconds, this basically means that you assume the current 
> second is always complete. This is only a problem (for a couple of seconds) 
> when all previous samples have zero activity or when the server is just 
> starting up.
> 
> Jay Kreps wrote:
> Actually there is a fundamental issue with the computation that patching 
> around the 0 case doesn't fix. That is the instability of the estimate. This 
> is the "taking a poll with sample size one" problem.
> 
> Even if you patch the 0 case you still get a bad answer 1 ms later. That 
> is, let's say you get a single 50k request and your quota is 1MB/sec. 
> Currently at 0ms we estimate infinity which is in fact the measured rate but 
> obviously not a good estimate. But even 1 ms later the estimate is bad. 
> 50k*1000ms = ~50MB/sec.
> 
> This is somewhat rare because it only happens when there is just one 
> sample.
> 
> They key observation is that if a sample is missing, nothing happened in 
> that time period. But the calculation should still use that time period.
> 
> So the right way to compute it, I think, is actually
>   ellapsed = (num_samples-1)*window_size + (now - current_sample.begin)
>   
> For safety I think we should also require the number of samples to be >= 
> 2 and default it to 3.

Hi Jay: thanks for comments!

I think the problem here is that, when event number is very small (e.g. 0, 1), 
what value should rate.measure() return, right? If I understand your solution 
formula right, when there is only one sample, your measured rate is n/ellapsed 
= n/(now - current_sample.begin), which is exactly same as the current code. I 
agree that requiring number of samples to be >= 2 solve the problem. But what 
happens when user call rate.measure() when there is only 1 sample?

I agree with you that we still get a bad answer if we patch the 0 case. How 
about we patch the 0 to timeWindowMs case: if (ellapsed < timeWindowMs) then 
ellapsed = timeWindowMs. Does this solve the problem here?


- Dong


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


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps


> On May 13, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 
> > 62
> > 
> >
> > Is this actually right? I agree you'll get discontinuities as the 
> > measured time shrinks to zero but why is giving back 0 the right answer? In 
> > the case you guys were testing 0 was safe, but imagine a case where the 
> > monitoring was checking that the value didn't fall below some threshold.
> 
> Dong Lin wrote:
> The question is, when Rate.measure() is called right after 
> Rate.record(n), what should be the return value? I think there are two 
> possibilities: 0 and n/config.timeWindowMs(). I didn't find any use case 
> where these two values make a difference. 
> 
> Which value do you think is the best?
> 
> Thank you.
> 
> Aditya Auradkar wrote:
> I think returning 0 is reasonable if no time has elapsed (technically).
> As an alternate solution, what if we assumed that "elapsed" is always 1 
> (at least). For example:
> 
> double elapsed = convert(now - stat.oldest(now).lastWindowMs) + 1
> 
> In case of seconds, this basically means that you assume the current 
> second is always complete. This is only a problem (for a couple of seconds) 
> when all previous samples have zero activity or when the server is just 
> starting up.

Actually there is a fundamental issue with the computation that patching around 
the 0 case doesn't fix. That is the instability of the estimate. This is the 
"taking a poll with sample size one" problem.

Even if you patch the 0 case you still get a bad answer 1 ms later. That is, 
let's say you get a single 50k request and your quota is 1MB/sec. Currently at 
0ms we estimate infinity which is in fact the measured rate but obviously not a 
good estimate. But even 1 ms later the estimate is bad. 50k*1000ms = ~50MB/sec.

This is somewhat rare because it only happens when there is just one sample.

They key observation is that if a sample is missing, nothing happened in that 
time period. But the calculation should still use that time period.

So the right way to compute it, I think, is actually
  ellapsed = (num_samples-1)*window_size + (now - current_sample.begin)
  
For safety I think we should also require the number of samples to be >= 2 and 
default it to 3.


- Jay


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


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps


> 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
> > 
> >
> > 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.

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?


- Jay


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


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps


> 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
> > 
> >
> > 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.

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


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


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Aditya Auradkar

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



clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java


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.


- Aditya Auradkar


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Aditya Auradkar


> On May 13, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 
> > 62
> > 
> >
> > Is this actually right? I agree you'll get discontinuities as the 
> > measured time shrinks to zero but why is giving back 0 the right answer? In 
> > the case you guys were testing 0 was safe, but imagine a case where the 
> > monitoring was checking that the value didn't fall below some threshold.
> 
> Dong Lin wrote:
> The question is, when Rate.measure() is called right after 
> Rate.record(n), what should be the return value? I think there are two 
> possibilities: 0 and n/config.timeWindowMs(). I didn't find any use case 
> where these two values make a difference. 
> 
> Which value do you think is the best?
> 
> Thank you.

I think returning 0 is reasonable if no time has elapsed (technically).
As an alternate solution, what if we assumed that "elapsed" is always 1 (at 
least). For example:

double elapsed = convert(now - stat.oldest(now).lastWindowMs) + 1

In case of seconds, this basically means that you assume the current second is 
always complete. This is only a problem (for a couple of seconds) when all 
previous samples have zero activity or when the server is just starting up.


- Aditya


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


On May 13, 2015, 10:32 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 10:32 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Dong Lin

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

(Updated May 13, 2015, 10:32 p.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 (updated)
-

  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



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Dong Lin


> On May 13, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java,
> >  line 126
> > 
> >
> > how come we are removing the event count threshold.

Oh.. My bad, I should not remove this one.

When I tried to interpret the semantics of SampledStat.measure(), I was 
confused by the use of eventCount -- I could not understand the meaning of 
Rate.measure() when both timeWindowMs and eventWindow are used in 
Sample.isComplete(). And I find that, except for the unit tests, 
config.eventWindow is always LONG.MAX_VALUE except. I removed it for these two 
reasons.

Now I understand there is use case to use eventWindow to expire sample. Though 
I still couldn't understand why user will use both timeWindowMs and eventWindow 
to expire sample.


> On May 13, 2015, 5:14 p.m., Jay Kreps wrote:
> > clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java, line 
> > 62
> > 
> >
> > Is this actually right? I agree you'll get discontinuities as the 
> > measured time shrinks to zero but why is giving back 0 the right answer? In 
> > the case you guys were testing 0 was safe, but imagine a case where the 
> > monitoring was checking that the value didn't fall below some threshold.

The question is, when Rate.measure() is called right after Rate.record(n), what 
should be the return value? I think there are two possibilities: 0 and 
n/config.timeWindowMs(). I didn't find any use case where these two values make 
a difference. 

Which value do you think is the best?

Thank you.


- Dong


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


On May 13, 2015, 3:47 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 3:47 p.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
> 
>



Re: Review Request 34170: Patch for KAFKA-2191

2015-05-13 Thread Jay Kreps

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



clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java


Is this actually right? I agree you'll get discontinuities as the measured 
time shrinks to zero but why is giving back 0 the right answer? In the case you 
guys were testing 0 was safe, but imagine a case where the monitoring was 
checking that the value didn't fall below some threshold.



clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java


how come we are removing the event count threshold.


- Jay Kreps


On May 13, 2015, 3:47 p.m., Dong Lin wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/34170/
> ---
> 
> (Updated May 13, 2015, 3:47 p.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
> 
>