Re: Stuck consumer with new consumer API in 0.9

2016-01-26 Thread Rajiv Kurian
Thanks Jun. On Tue, Jan 26, 2016 at 3:48 PM, Jun Rao wrote: > Rajiv, > > We haven't released 0.9.0.1 yet. To try the fix, you can build a new client > jar off the 0.9.0 branch. > > Thanks, > > Jun > > On Mon, Jan 25, 2016 at 12:03 PM, Rajiv Kurian wrote: > > > Thanks Jason. We are using an affe

Re: Stuck consumer with new consumer API in 0.9

2016-01-26 Thread Jun Rao
Rajiv, We haven't released 0.9.0.1 yet. To try the fix, you can build a new client jar off the 0.9.0 branch. Thanks, Jun On Mon, Jan 25, 2016 at 12:03 PM, Rajiv Kurian wrote: > Thanks Jason. We are using an affected client I guess. > > Is there a 0.9.0 client available on maven? My search at

Re: Stuck consumer with new consumer API in 0.9

2016-01-26 Thread Bruno Rassaerts
We do not seek in the onPartitionAssigned. In our test setup (evaluating kafka for a new project) we put a constant load on one of the topics. We have a consumer group pulling messages from the different partitions on the topic. At a certain point in time, the poll() does not return any message

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Han JU
Hi Guozhang, Sorry for that example. They does not come from the same run, just paste that to illustrate the problem. I'll try out what Jason suggests tomorrow and also retry the 0.9.0 branch. 2016-01-25 21:03 GMT+01:00 Rajiv Kurian : > Thanks Jason. We are using an affected client I guess. > >

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Rajiv Kurian
Thanks Jason. We are using an affected client I guess. Is there a 0.9.0 client available on maven? My search at http://mvnrepository.com/artifact/org.apache.kafka/kafka_2.10 only shows the 0.9.0.0 client which seems to have this issue. Thanks, Rajiv On Mon, Jan 25, 2016 at 11:56 AM, Jason Gusta

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Jason Gustafson
Hey Rajiv, the bug was on the client. Here's a link to the JIRA: https://issues.apache.org/jira/browse/KAFKA-2978. -Jason On Mon, Jan 25, 2016 at 11:42 AM, Rajiv Kurian wrote: > Hi Jason, > > Was this a server bug or a client bug? > > Thanks, > Rajiv > > On Mon, Jan 25, 2016 at 11:23 AM, Jason

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Guozhang Wang
Han, >From your logs it seems the thread which cannot fetch more data is rebalance-1, which is assigned with partitions [balance-11, balance-10, balance-9]; >From your consumer-group command the partitions that is lagging are [balance-6, balance-7, balance-8] which is not assigned to this process

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Rajiv Kurian
Hi Jason, Was this a server bug or a client bug? Thanks, Rajiv On Mon, Jan 25, 2016 at 11:23 AM, Jason Gustafson wrote: > Apologies for the late arrival to this thread. There was a bug in the > 0.9.0.0 release of Kafka which could cause the consumer to stop fetching > from a partition after a

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Jason Gustafson
Apologies for the late arrival to this thread. There was a bug in the 0.9.0.0 release of Kafka which could cause the consumer to stop fetching from a partition after a rebalance. If you're seeing this, please checkout the 0.9.0 branch of Kafka and see if you can reproduce this problem. If you can,

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Ismael Juma
Thanks! Ismael On Mon, Jan 25, 2016 at 4:03 PM, Han JU wrote: > Issue created: https://issues.apache.org/jira/browse/KAFKA-3146 > > 2016-01-25 16:07 GMT+01:00 Han JU : > > > Hi Bruno, > > > > Can you tell me a little bit more about that? A seek() in the > > `onPartitionAssigned`? > > > > Thanks

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Han JU
Issue created: https://issues.apache.org/jira/browse/KAFKA-3146 2016-01-25 16:07 GMT+01:00 Han JU : > Hi Bruno, > > Can you tell me a little bit more about that? A seek() in the > `onPartitionAssigned`? > > Thanks. > > 2016-01-25 10:51 GMT+01:00 Han JU : > >> Ok I'll create a JIRA issue on this.

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Han JU
Hi Bruno, Can you tell me a little bit more about that? A seek() in the `onPartitionAssigned`? Thanks. 2016-01-25 10:51 GMT+01:00 Han JU : > Ok I'll create a JIRA issue on this. > > Thanks! > > 2016-01-23 21:47 GMT+01:00 Bruno Rassaerts : > >> +1 here >> >> As a workaround we seek to the curren

Re: Stuck consumer with new consumer API in 0.9

2016-01-25 Thread Han JU
Ok I'll create a JIRA issue on this. Thanks! 2016-01-23 21:47 GMT+01:00 Bruno Rassaerts : > +1 here > > As a workaround we seek to the current offset which resets the current > clients internal states and everything continues. > > Regards, > Bruno Rassaerts | Freelance Java Developer > > Novazon

Re: Stuck consumer with new consumer API in 0.9

2016-01-23 Thread Bruno Rassaerts
+1 here As a workaround we seek to the current offset which resets the current clients internal states and everything continues. Regards, Bruno Rassaerts | Freelance Java Developer Novazone, Edingsesteenweg 302, B-1755 Gooik, Belgium T: +32(0)54/26.02.03 - M:+32(0)477/39.01.15 bruno.rassae...@

Re: Stuck consumer with new consumer API in 0.9

2016-01-23 Thread Ismael Juma
Hi, Can you please file an issue in JIRA so that we make sure this is investigated? Ismael On Fri, Jan 22, 2016 at 3:13 PM, Han JU wrote: > Hi, > > I'm prototyping with the new consumer API of kafka 0.9 and I'm particularly > interested in the `ConsumerRebalanceListener`. > > My test setup is

Re: Stuck consumer with new consumer API in 0.9

2016-01-23 Thread 何伟昌
+1, facing same issue. > 在 2016年1月22日,下午11:13,Han JU 写道: > > Hi, > > I'm prototyping with the new consumer API of kafka 0.9 and I'm particularly > interested in the `ConsumerRebalanceListener`. > > My test setup is like the following: > - 5M messages pre-loaded in one node kafka 0.9 > - 12 p

Stuck consumer with new consumer API in 0.9

2016-01-22 Thread Han JU
Hi, I'm prototyping with the new consumer API of kafka 0.9 and I'm particularly interested in the `ConsumerRebalanceListener`. My test setup is like the following: - 5M messages pre-loaded in one node kafka 0.9 - 12 partitions, auto offset commit set to false - in `onPartitionsRevoked`, com