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

Jason Rosenberg commented on KAFKA-1029:
----------------------------------------

Just to wrap this up, it turns out the app that was seeing this behavior had a 
but which was causing unnecessary memory pressure and continual garbage 
collection.  This ended up slowing the app down, such that zk timeouts were 
continual, and the consumer connector decided continually to keep re-balancing. 
 This caused the flapping of ephemeral nodes, and other problems, it would seem 
(the last part here is conjecture).

So, I think the app was the problem, not the high-level kafka consumer code.  
However, perhaps there is something that can be done to prevent or throttle 
continual re-balancing in this case!  It was confusing to look at the logs and 
see almost exclusively warnings/exceptions coming from kafka, and not the app 
itself.

> Zookeeper leader election stuck in ephemeral node retry loop
> ------------------------------------------------------------
>
>                 Key: KAFKA-1029
>                 URL: https://issues.apache.org/jira/browse/KAFKA-1029
>             Project: Kafka
>          Issue Type: Bug
>          Components: controller
>    Affects Versions: 0.8.0
>            Reporter: Sam Meder
>            Assignee: Sam Meder
>            Priority: Blocker
>             Fix For: 0.8.0
>
>         Attachments: 
> 0002-KAFKA-1029-Use-brokerId-instead-of-leaderId-when-tri.patch
>
>
> We're seeing the following log statements (over and over):
> [2013-08-27 07:21:49,538] INFO conflict in /controller data: { "brokerid":3, 
> "timestamp":"1377587945206", "version":1 } stored data: { "brokerid":2, 
> "timestamp":"1377587460904", "version":1 } (kafka.utils.ZkUtils$)
> [2013-08-27 07:21:49,559] INFO I wrote this conflicted ephemeral node [{ 
> "brokerid":3, "timestamp":"1377587945206", "version":1 }] at /controller a 
> while back in a different session, hence I will backoff for this node to be 
> deleted by Zookeeper and retry (kafka.utils.ZkUtils$)
> where the broker is essentially stuck in the loop that is trying to deal with 
> left-over ephemeral nodes. The code looks a bit racy to me. In particular:
> ZookeeperLeaderElector:
>   def elect: Boolean = {
>     controllerContext.zkClient.subscribeDataChanges(electionPath, 
> leaderChangeListener)
>     val timestamp = SystemTime.milliseconds.toString
>     val electString = ...
>     try {
>       
> createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, 
> electionPath, electString, leaderId,
>         (controllerString : String, leaderId : Any) => 
> KafkaController.parseControllerId(controllerString) == 
> leaderId.asInstanceOf[Int],
>         controllerContext.zkSessionTimeout)
> leaderChangeListener is registered before the create call (by the way, it 
> looks like a new registration will be added every elect call - shouldn't it 
> register in startup()?) so can update leaderId to the current leader before 
> the call to create. If that happens then we will continuously get node exists 
> exceptions and the checker function will always return true, i.e. we will 
> never get out of the while(true) loop.
> I think the right fix here is to pass brokerId instead of leaderId when 
> calling create, i.e.
> createEphemeralPathExpectConflictHandleZKBug(controllerContext.zkClient, 
> electionPath, electString, brokerId,
>         (controllerString : String, leaderId : Any) => 
> KafkaController.parseControllerId(controllerString) == 
> leaderId.asInstanceOf[Int],
>         controllerContext.zkSessionTimeout)
> The loop dealing with the ephemeral node bug is now only triggered for the 
> broker that owned the node previously, although I am still not 100% sure if 
> that is sufficient.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to