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

Flavio Junqueira commented on KAFKA-3042:
-----------------------------------------

[~junrao] It makes sense, thanks for the analysis. Trying to reconstruct the 
problem in steps, this is what's going on:

# Broker 5 thinks broker 4 is alive and sends a LeaderAndIsr request to broker 
1 with 4 as the leader.
# Broker 1 doesn't have 4 cached as a live broker, so it fails the request to 
make it a follower of the partition.

The LeaderAndIsr request has a list of live leaders, and I suppose 4 is in that 
list. 

To sort this out, I can see two options:

# We simply update the metadata cache upon receiving a LeaderAndIsr request 
using the list of live leaders. This update needs to be  the union of the 
current set with the set of leaders.
# You also suggested to send an UpdateMetadata request first to update the set 
of love brokers. 

I can't see any problem with 1, and I can't see any immediate problem with 2 
either, but I'm concerned about finding ourselves with another race condition 
if we send an update first. What do you think?  

> updateIsr should stop after failed several times due to zkVersion issue
> -----------------------------------------------------------------------
>
>                 Key: KAFKA-3042
>                 URL: https://issues.apache.org/jira/browse/KAFKA-3042
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.2.1
>         Environment: jdk 1.7
> centos 6.4
>            Reporter: Jiahongchao
>             Fix For: 0.10.0.0
>
>         Attachments: controller.log, server.log.2016-03-23-01, 
> state-change.log
>
>
> sometimes one broker may repeatly log
> "Cached zkVersion 54 not equal to that in zookeeper, skip updating ISR"
> I think this is because the broker consider itself as the leader in fact it's 
> a follower.
> So after several failed tries, it need to find out who is the leader



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to