Actually, I saw this line in the log : can't rebalance after 4 retries.
What should I expect in this case? All consumers threads failed or only some of 
them?
If I increase the number of retries or delay between retries, will that help?

Regards,

Libo


-----Original Message-----
From: Jun Rao [mailto:jun...@gmail.com] 
Sent: Friday, November 29, 2013 8:50 PM
To: users@kafka.apache.org
Subject: Re: ConsumerRebalanceFailedException

Transient rebalance failures are ok. However, it's important that the last 
rebalance in a sequence succeeds. Otherwise, some of the partitions will not be 
consumed by any consumers.

Thanks,

Jun


On Fri, Nov 29, 2013 at 10:44 AM, Yu, Libo <libo...@citi.com> wrote:

> You are right, Joe. I checked our brokers' log. We have three brokers. 
> All of them failed to connect to zk at some point.
> So they were offline and later reregistered themselves with the zk. I 
> don't know how many rebalance should be triggered in that case. There 
> is only one exception found in consumer's log. My question is whether 
> users need to do anything to handle ConsumerRebalanceFailedException.
>
> This is from consumer log:
>
> [28/11/13 16:38:56:056 PM EST] 102 ERROR
> consumer.ZookeeperConsumerConnector: [xxxxxxxxxx ], error during 
> syncedRebalance
> kafka.common.ConsumerRebalanceFailedException: xxxxxxxxx can't 
> rebalance after 4 retries
>         at
> kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener.syncedR
> eb
> alance(ZookeeperConsumerConnector.scala:397)
>         at
> kafka.consumer.ZookeeperConsumerConnector$ZKRebalancerListener$$anon$1
> .r
> un(ZookeeperConsumerConnector.scala:326)
>
> Regards,
>
> Libo
>
>
> -----Original Message-----
> From: Joe Stein [mailto:joe.st...@stealth.ly]
> Sent: Friday, November 29, 2013 11:57 AM
> To: users@kafka.apache.org
> Subject: Re: ConsumerRebalanceFailedException
>
> What is the full stack trace?  if you see "can't rebalance after 4 retries"
> then likely the problem is the broker is down or not available
>
> /*******************************************
>  Joe Stein
>  Founder, Principal Consultant
>  Big Data Open Source Security LLC
>  http://www.stealth.ly
>  Twitter: @allthingshadoop <http://www.twitter.com/allthingshadoop>
> ********************************************/
>
>
> On Fri, Nov 29, 2013 at 11:31 AM, Yu, Libo <libo...@citi.com> wrote:
>
> > We found our consumer stopped working after this exception occurred.
> > Can the consumer recover from such an exception?
> >
> > Regards,
> >
> > Libo
> >
> >
> > -----Original Message-----
> > From: Florin Trofin [mailto:ftro...@adobe.com]
> > Sent: Tuesday, July 16, 2013 4:20 PM
> > To: users@kafka.apache.org
> > Subject: Re: ConsumerRebalanceFailedException
> >
> > Yes, I think these are two separate issues.
> >
> > F.
> >
> > On 7/16/13 11:32 AM, "Joel Koshy" <jjkosh...@gmail.com> wrote:
> >
> > >From a user's perspective, ConsumerRebalanceException is a bit 
> > >cryptic -I think the other thread was to provide a more informative 
> > >message and also be able to recover when a broker does come up 
> > >(fixed in KAFKA-969).
> > >
> > >Thanks,
> > >
> > >Joel
> > >
> > >On Tue, Jul 16, 2013 at 11:04 AM, Vaibhav Puranik 
> > ><vpura...@gmail.com>
> > >wrote:
> > >> Thank you Joel.
> > >>
> > >> In a different but related thread, somebody is asking to rename 
> > >> the exception as NoBrokerAvailableExcption. But given the 
> > >> description above, the exception seems to be named appropriately.
> > >>
> > >> Regards,
> > >> Vaibhav
> > >>
> > >>
> > >> On Tue, Jul 16, 2013 at 12:05 AM, Joel Koshy 
> > >><jjkosh...@gmail.com>
> > >>wrote:
> > >>
> > >>> Yes - rebalance => consumers trying to coordinate through ZK.
> > >>> Rebalances can happen when one or more of the following happen:
> > >>> - a consumed topic partition appears or disappears - i.e., if a 
> > >>> broker comes or goes.
> > >>> - a consumer instance in the group comes or goes "goes" could 
> > >>> also be triggered by session expirations in zookeeper - 
> > >>> typically caused by client-side GC or flaky connections to zookeeper.
> > >>>
> > >>> On Mon, Jul 15, 2013 at 10:15 AM, Vaibhav Puranik 
> > >>> <vpura...@gmail.com>
> > >>> wrote:
> > >>> > Hi all,
> > >>> >
> > >>> > We have a small Kafka cluster (0.7.1 - 3 nodes) in EC2. The 
> > >>> > load is
> > >>>about
> > >>> > 200 million events per day, each being few kilobytes. We have 
> > >>> > a
> > >>>single
> > >>> node
> > >>> > zookeeper.
> > >>> >
> > >>> > Yesterday suddenly our Kafka clients started throwing the 
> > >>> > following
> > >>> > exception:
> > >>> > java.lang.RuntimeException:
> > >>> kafka.common.ConsumerRebalanceFailedException:
> > >>> >
> > >>>CONSUMER_GROUP_NAME_ip-00-00-00-00.ec2.internal-1373821190828-5f7
> > >>>8e
> > >>>9a
> > >>>f
> > >>> > can't rebalance after 4 retries
> > >>> >     at
> > >>> >
> > >>>
> > >>>com.gumgum.kafka.consumer.KafkaTemplate.executeWithBatch(KafkaTem
> > >>>pl
> > >>>at
> > >>>e.j
> > >>>ava:59)
> > >>> >     at
> > >>> >
> > >>>
> > >>>com.gumgum.storm.fileupload.GenericKafkaSpout.nextTuple(GenericKa
> > >>>fk
> > >>>aS
> > >>>pou
> > >>>t.java:73)
> > >>> >     at
> > >>> >
> > >>>
> > >>>backtype.storm.daemon.executor$fn__3968$fn__4009$fn__4010.invoke(
> > >>>ex
> > >>>ec
> > >>>uto
> > >>>r.clj:433)
> > >>> >     at
> > >>> > backtype.storm.util$async_loop$fn__465.invoke(util.clj:377)
> > >>> >
> > >>> > None of the Kafka clients (ConsumerConenctor class) would start.
> > >>> > They
> > >>> would
> > >>> > fail with the exception.
> > >>> >
> > >>> > We tried restarting the clilents, restarting the zookeeper as well.
> > >>>But
> > >>> > finally it all started working when we restarted all of our 
> > >>> > kafka
> > >>> brokers.
> > >>> > We didn't lose any data because producers (going directly to 
> > >>> > the
> > >>>brokers
> > >>> > through a load balancer) were working fine.
> > >>> >
> > >>> > I tried googling this issue and looks like lot of people have 
> > >>> > faced
> > >>>it,
> > >>> but
> > >>> > couldn't get anything concrete.
> > >>> >
> > >>> > Given this, I have two questions:
> > >>> >
> > >>> > It will be nice if you can tell me why this can happen or 
> > >>> > point me
> > >>>to a
> > >>> > link where I can understand it better. What does Consumer 
> > >>> > Rebalancing
> > >>> mean?
> > >>> > Does that mean consumers are trying to coordinate amongst 
> > >>> > themselves
> > >>> using
> > >>> > Zookeeper?
> > >>> >
> > >>> > On a separate note, are there any JMX parameters I need to be
> > >>>monitoring
> > >>> to
> > >>> > make sure that my kafka cluster is healthy? How can I keep 
> > >>> > watch on
> > >>>my
> > >>> > kafka cluster?
> > >>> >
> > >>> > Regards,
> > >>> > Vaibhav Puranik
> > >>> > GumGum
> > >>>
> >
> >
>

Reply via email to