Thanks Guozhang for the detailed explanation.
It was really helpful.

Regards,
Rahul Misra


-----Original Message-----
From: Guozhang Wang [mailto:wangg...@gmail.com] 
Sent: Wednesday, October 12, 2016 6:25 AM
To: users@kafka.apache.org
Subject: Re: Frequent Consumer Rebalance/ Commit fail exception

Hello Rahul,

This "CommitFailedException" usually means that the consumer group coordinator 
that sits on the server side has decided that this consumer is "failed" from 
the heartbeat protocol and hence kicked out of the group, and later when it 
sees a commit-offset request from this consumer it will just reject the request 
with "rebalance in progress" error code.

Since you mentioned that you still periodically call poll just toe send the 
heartbeat I'd suspect this is because the heartbeat is not somehow sent in 
time. Consider either increase the polling frequency while pausing the topics, 
or set the "session.timeout.ms" value larger so that it is less sensitive to 
heartbeat delays.


BTW, in the up-coming 0.10.1.0 release a new feature (KIP-62:
https://cwiki.apache.org/confluence/display/KAFKA/KIP-62%3A+Allow+consumer+to+send+heartbeats+from+a+background+thread)
is added to the consumer, where session timeout is separate from a new config 
named "processing.timeout.ms" so that you can set the latter config to be much 
higher (default value is like 5 minutes) so that the consume-then-process 
pattern can be more conveniently supported than pause-and-poll pattern. Hope it 
helps.


Guozhang


On Mon, Oct 10, 2016 at 12:13 PM, Misra, Rahul <rahul.mi...@altisource.com>
wrote:

> Hi,
>
> I have a custom Kafka consumer which reads messages from a topic, 
> hands over the processing of the messages  to a different thread, and 
> while the messages are being processed, it pauses the topic and keeps 
> polling the Kafka topic (to maintain heartbeats) and also commits 
> offsets using
> commitSync() once the processing thread returns success.
> This consumer is the only consumer in its group. Auto commit for 
> offsets is set to false.
>
> The consumer also registers the onPartitionsAssigned() and
> onPartitionsRevoked() listeners.
>
> Recently I observed that the consumer frequently crashes (if consuming 
> large number of messages) with the following exception:
>
>
>         org.apache.kafka.clients.consumer.CommitFailedException: 
> Commit cannot be completed due to group rebalance at org.apache.kafka.clients.
> consumer.internals.ConsumerCoordinator$OffsetCommitResponseHandler.
> handle(ConsumerCoordinator.java:546)
>         at org.apache.kafka.clients.consumer.internals.
> ConsumerCoordinator$OffsetCommitResponseHandler.
> handle(ConsumerCoordinator.java:487)
>         at org.apache.kafka.clients.consumer.internals.
> AbstractCoordinator$CoordinatorResponseHandler.
> onSuccess(AbstractCoordinator.java:681)
>         at org.apache.kafka.clients.consumer.internals.
> AbstractCoordinator$CoordinatorResponseHandler.
> onSuccess(AbstractCoordinator.java:654)
>         at org.apache.kafka.clients.consumer.internals.
> RequestFuture$1.onSuccess(RequestFuture.java:167)
>         at org.apache.kafka.clients.consumer.internals.
> RequestFuture.fireSuccess(RequestFuture.java:133)
>         at org.apache.kafka.clients.consumer.internals.
> RequestFuture.complete(RequestFuture.java:107)
>         at org.apache.kafka.clients.consumer.internals.
> ConsumerNetworkClient$RequestFutureCompletionHandler.onComplete(
> ConsumerNetworkClient.java:350)
>         at org.apache.kafka.clients.NetworkClient.poll(
> NetworkClient.java:288)
>         at org.apache.kafka.clients.consumer.internals.
> ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:303)
>         at org.apache.kafka.clients.consumer.internals.
> ConsumerNetworkClient.poll(ConsumerNetworkClient.java:197)
>         at org.apache.kafka.clients.consumer.internals.
> ConsumerNetworkClient.poll(ConsumerNetworkClient.java:187)
>
> None of the rebalance listeners were called before this exception.
> Could somebody suggest why this rebalancing is being triggered, always 
> while committing the offsets (or is the actual issue somewhere else?)
>
> Regards,
> Rahul Misra
>
> This email message and any attachments are intended solely for the use 
> of the addressee. If you are not the intended recipient, you are 
> prohibited from reading, disclosing, reproducing, distributing, 
> disseminating or otherwise using this transmission. If you have 
> received this message in error, please promptly notify the sender by 
> reply email and immediately delete this message from your system. This 
> message and any attachments may contain information that is 
> confidential, privileged or exempt from disclosure. Delivery of this 
> message to any person other than the intended recipient is not 
> intended to waive any right or privilege. Message transmission is not 
> guaranteed to be secure or free of software viruses.
> ************************************************************
> ***********************************************************
>



--
-- Guozhang
This email message and any attachments are intended solely for the use of the 
addressee. If you are not the intended recipient, you are prohibited from 
reading, disclosing, reproducing, distributing, disseminating or otherwise 
using this transmission. If you have received this message in error, please 
promptly notify the sender by reply email and immediately delete this message 
from your system. This message and any attachments may contain information that 
is confidential, privileged or exempt from disclosure. Delivery of this message 
to any person other than the intended recipient is not intended to waive any 
right or privilege. Message transmission is not guaranteed to be secure or free 
of software viruses. 
***********************************************************************************************************************

Reply via email to