[jira] [Commented] (KAFKA-2486) New consumer performance

2015-09-01 Thread Jason Gustafson (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14726063#comment-14726063
 ] 

Jason Gustafson commented on KAFKA-2486:


[~jkreps] I've been running the consumer performance tests locally and the 
performance is similar to that of the old consumer. The initial rebalance seems 
to skew the results for small tests (i.e. with fewer messages), but the results 
appear to converge for larger tests. I also noticed significantly lower cpu 
utilization for the new consumer. It would be nice if others could confirm this.

> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-09-01 Thread Ben Stopford (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14726085#comment-14726085
 ] 

Ben Stopford commented on KAFKA-2486:
-

[~hachikuji] [~jkreps] I'd add that the new consumer handles smaller messages 
better than the old one, at least with default settings. Larger messages show 
pretty equivalent results. 

> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-31 Thread Jason Gustafson (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14723607#comment-14723607
 ] 

Jason Gustafson commented on KAFKA-2486:


The sleep was intended to implement the fetch backoff as Guozhang mentioned, 
but now that I think about it, the better place for it is inside Fetcher, which 
can track when new fetches need to be sent. I think Jay is right that we 
basically don't want to sleep ever since we could be missing IO.

> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-31 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14723615#comment-14723615
 ] 

Jay Kreps commented on KAFKA-2486:
--

I don't think we need to do anything special for the 0 wait time case. I think 
we can eventually add request count quotas to protect against this kind of 
thing, but for now I think maintaining the behavior of most the other consumers 
of "no backoff means no backoff" is totally fine. Presumably if you set that, 
that is what you want.

For the consumer benchmark case I don't think we need to override the max wait 
to 0, not sure why it was that way before. It should be fine with 500ms or 
whatever.

> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14723913#comment-14723913
 ] 

ASF GitHub Bot commented on KAFKA-2486:
---

GitHub user hachikuji opened a pull request:

https://github.com/apache/kafka/pull/180

KAFKA-2486; fix performance regression in new consumer

The sleep() in KafkaConsumer's poll blocked any pending IO from being 
completed and created a performance bottleneck. It was intended to implement 
the fetch backoff behavior, but that was a misunderstanding of the setting 
"retry.backoff.ms" which should only affect failed fetches. 

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/hachikuji/kafka KAFKA-2486

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/kafka/pull/180.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #180


commit 8bec099900fdb511cf9f4b5a31c63d695e6a9c49
Author: Jason Gustafson 
Date:   2015-08-31T19:23:30Z

KAFKA-2486; fix performance regression in new consumer




> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-31 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14724070#comment-14724070
 ] 

ASF GitHub Bot commented on KAFKA-2486:
---

Github user asfgit closed the pull request at:

https://github.com/apache/kafka/pull/180


> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-31 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14724079#comment-14724079
 ] 

Jay Kreps commented on KAFKA-2486:
--

Where does this leave perf for the new consumer vs old? Maybe that's getting 
looked into as part of the SSL testing...

> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-31 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14724448#comment-14724448
 ] 

Jay Kreps commented on KAFKA-2486:
--

[~ewencp] Nightly will be awesome!

I think what I am asking is more basic--did this change actually address the 
perf issue? I.e. is the new consumer and old consumer now back at parity? I ask 
because a lot of code changed since the one off testing so I wouldn't be 
shocked if two or three things all went bad.

> New consumer performance
> 
>
> Key: KAFKA-2486
> URL: https://issues.apache.org/jira/browse/KAFKA-2486
> Project: Kafka
>  Issue Type: Sub-task
>  Components: consumer
>Reporter: Ewen Cheslack-Postava
>Assignee: Jason Gustafson
> Fix For: 0.8.3
>
>
> The new consumer was previously reaching getting good performance. However, a 
> recent report on the mailing list indicates it's dropped significantly. After 
> evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
> compared to 600+MB/s previously. Before release, we should get the 
> performance back on par.
> Some details about where the regression occurred from the mailing list 
> http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
>  :
> bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
> performance -> an error due to broker apparently not accepting the partition 
> assignment strategy. Since this commit seems to add heartbeats and the server 
> side code for partition assignment strategies, I assume we were missing 
> something on the client side and by filling in the server side, things 
> stopped working.
> bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
> a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
> perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
> but that's still far less than the 600MB/s I saw on the earlier commits.
> Ideally we would also at least have a system test in place for the new 
> consumer, even if regressions weren't automatically detected. It would at 
> least allow for manually checking for regressions. This should not be 
> difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Ben Stopford (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14719974#comment-14719974
 ] 

Ben Stopford commented on KAFKA-2486:
-

Switching to:

  while(records.isEmpty)  records = consumer.poll(1)

speeds things up significantly simply because it avoids the sleep(100)


 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Ben Stopford (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14719295#comment-14719295
 ] 

Ben Stopford commented on KAFKA-2486:
-

The mail thread refers to the user of ConsumerPerformance. This in turn uses 
poll(100), as is suggested in our docs. 

I'm unclear as to whether this is a bug in the use of the API in 
ConsumerPerformance or a bug in the API itself.

The current behaviour is to sleep the timeout period on every poll if  (server 
fetch time)  (an iteration of the client's record processing loop). That's to 
say, we implement prefetching, but it's quite likely that the result won't be 
back in time if the client isn't making an independent network call or doing 
some pretty time consuming processing. 

Should we be waking the sleeping client thread when the pre-fetch returns 
results? 

 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14719979#comment-14719979
 ] 

Jay Kreps commented on KAFKA-2486:
--

Hmm, I think it should be 
{code}
while(records.isEmpty  remaining  0) {
   consumer.poll(min(remaining, retryBackoffMs));
   remaining -= time.milliseconds() - end;
}
{code}

 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14720050#comment-14720050
 ] 

Jay Kreps commented on KAFKA-2486:
--

[~guozhang] these are long-poll fetches so if there is not enough data the 
fetch waits in purgatory on the server and no new fetch will be issued, so not 
sure if I get the scenario you're trying to prevent. Can you elaborate?

 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14719963#comment-14719963
 ] 

Jay Kreps commented on KAFKA-2486:
--

Wait, we sleep? That is totally messed up, you can't sleep. You have to poll, 
which is the same as sleep except it stops when you have data. 

 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14720040#comment-14720040
 ] 

Guozhang Wang commented on KAFKA-2486:
--

The sleep is introduced in b6d326b0893e60b350608260fd1bd2542337cb5a as part of 
KAFKA-2168. The motivation was that when there is no enough data on the broker 
yet we should just treat it as failed fetch request and try to avoid 
bombarding the servers with continuous fetch requests. I think there is still 
value in backing off when there is no fetchable data, while the retry.backoff 
config's default value 100ms could be OK in some real production environments, 
for performance profiling it should really be 0.

 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Guozhang Wang (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14720110#comment-14720110
 ] 

Guozhang Wang commented on KAFKA-2486:
--

OK, there are a few mixed expressions in my previous comment. First of all 
retry.backoff should be used for failed requests, but not requests returning 
no data, so I did not catch that when reviewing and this needs to be fixed for 
sure. No question about it.

Now return to performance profiling, by default the fetch requests are 
long-polls:

FETCH_MIN_BYTES_CONFIG = 1024
FETCH_MAX_WAIT_MS_CONFIG = 500

but I remember in consumer performance they were once overriden to 1 and 0 (I 
saw those overriden values were removed now, but I am pretty confident they 
were over-written some time ago), in which case we may want to back-off a bit 
instead of DDoSing. The question is really for performance profiling where data 
keeps filling in from a continuously sending producer and there is no 
processing time after consumption, what should be the optimal configs. I think 
long polls should still be preferable for throughput, but maybe will give 
sub-optimal latencies.

 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (KAFKA-2486) New consumer performance

2015-08-28 Thread Jay Kreps (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-2486?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14720237#comment-14720237
 ] 

Jay Kreps commented on KAFKA-2486:
--

[~guozhang] I interpret you to be saying that even if there is data, sometimes 
fetching immediately isn't ideal for throughput because of the reduction in 
batching? I think that is right but that is controlled by the min bytes setting 
in the fetch request, right? In other words I think we already address that. 
Not sure what the ideal consumer perf settings are. I think for that perf test 
it shoudn't matter too much what the max wait is because there is always data 
to fetch so it never really get's used.

I agree that right now letting the consumer set the max wait to 0 does allow 
the possibility of abuse but I think the fix for that is probably an eventual 
server-side quota.

 New consumer performance
 

 Key: KAFKA-2486
 URL: https://issues.apache.org/jira/browse/KAFKA-2486
 Project: Kafka
  Issue Type: Sub-task
  Components: consumer
Reporter: Ewen Cheslack-Postava
Assignee: Jason Gustafson
 Fix For: 0.8.3


 The new consumer was previously reaching getting good performance. However, a 
 recent report on the mailing list indicates it's dropped significantly. After 
 evaluation, even with a local broker it seems to only be reaching a 2-10MB/s, 
 compared to 600+MB/s previously. Before release, we should get the 
 performance back on par.
 Some details about where the regression occurred from the mailing list 
 http://mail-archives.apache.org/mod_mbox/kafka-dev/201508.mbox/%3CCAAdKFaE8bPSeWZf%2BF9RuA-xZazRpBrZG6vo454QLVHBAk_VOJg%40mail.gmail.com%3E
  :
 bq. At 49026f11781181c38e9d5edb634be9d27245c961 (May 14th), we went from good 
 performance - an error due to broker apparently not accepting the partition 
 assignment strategy. Since this commit seems to add heartbeats and the server 
 side code for partition assignment strategies, I assume we were missing 
 something on the client side and by filling in the server side, things 
 stopped working.
 bq. On either 84636272422b6379d57d4c5ef68b156edc1c67f8 or 
 a5b11886df8c7aad0548efd2c7c3dbc579232f03 (July 17th), I am able to run the 
 perf test again, but it's slow -- ~10MB/s for me vs the 2MB/s Jay was seeing, 
 but that's still far less than the 600MB/s I saw on the earlier commits.
 Ideally we would also at least have a system test in place for the new 
 consumer, even if regressions weren't automatically detected. It would at 
 least allow for manually checking for regressions. This should not be 
 difficult since there are already old consumer performance tests.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)