Re: Getting very poor performance from the new Kafka consumer

2016-01-27 Thread Rajiv Kurian
Hi Jason, Thanks for investigating. Indeed we do have probably more than the usual number of partitions. Our use case is such that we have many partitions (128 - 256) overall but very few messages per second on each partition. I have created a JIRA at https://issues.apache.org/jira/browse/KAFKA-3

Re: Getting very poor performance from the new Kafka consumer

2016-01-27 Thread Jason Gustafson
Hey Rajiv, Thanks for the detailed report. Can you go ahead and create a JIRA? I do see the exceptions locally, but not nearly at the rate that you're reporting. That might be a factor of the number of partitions, so I'll do some investigation. -Jason On Wed, Jan 27, 2016 at 8:40 AM, Rajiv Kuria

Re: Getting very poor performance from the new Kafka consumer

2016-01-27 Thread Rajiv Kurian
Hi Guozhang, The Github link I pasted was from the 0.9.0 branch. The same line seems to be throwing exceptions in my code built of the maven 0.9.0.0 package. Are you saying that something else has changed higher up the call stack that will probably not trigger so many exceptions ? Thanks, Rajiv

Re: Getting very poor performance from the new Kafka consumer

2016-01-26 Thread Guozhang Wang
Rajiv, Could you try to build the new consumer from 0.9.0 branch and see if the issue can be re-produced? Guozhang On Mon, Jan 25, 2016 at 9:46 PM, Rajiv Kurian wrote: > The exception seems to be thrown here > > https://github.com/apache/kafka/blob/0.9.0/clients/src/main/java/org/apache/kafka/

Re: Getting very poor performance from the new Kafka consumer

2016-01-25 Thread Rajiv Kurian
The exception seems to be thrown here https://github.com/apache/kafka/blob/0.9.0/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java#L236 Is this not expected to hit often? On Mon, Jan 25, 2016 at 9:22 PM, Rajiv Kurian wrote: > Wanted to add that we are not using auto commit

Re: Getting very poor performance from the new Kafka consumer

2016-01-25 Thread Rajiv Kurian
Wanted to add that we are not using auto commit since we use custom partition assignments. In fact we never call consumer.commitAsync() or consumer.commitSync() calls. My assumption is that since we store our own offsets these calls are not necessary. Hopefully this is not responsible for the poor

Getting very poor performance from the new Kafka consumer

2016-01-25 Thread Rajiv Kurian
We are using the new kafka consumer with the following config (as logged by kafka) metric.reporters = [] metadata.max.age.ms = 30 value.deserializer = class org.apache.kafka.common.serialization.ByteArrayDeserializer group.id = myGroup.id partition.assignmen