[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2017-04-19 Thread Jeff Widman (JIRA)

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

Jeff Widman commented on KAFKA-3042:


[~lindong] thanks for the offer to help and sorry for the slow response. 

I'm not exactly sure how to repro, but below I copied a sanitized version of 
our internal wiki page documenting our findings as we tried to figure out what 
was happening and how we got into the state of mis-matched controller epoch for 
controller vs random partition. It's not the most polished, more of a train of 
thought put to paper as we debugged.

Reading through it, it appeared that broker 3 lost connection to zookeeper, 
then when it came back, it elected itself controller, but somehow ended up in a 
state where the broker 3 controller had a list of brokers that was completely 
empty. This doesn't make logical sense because if a broker is controller, then 
it should list itself in active brokers. But somehow it happened. Then 
following that, the active epoch for the controller is 134, but the active 
epoch listed by a random partition in zookeeper is 133. So that created the 
version mismatch. 

More details below, and I also have access to the detailed Kafka logs (but not 
ZK logs) beyond just the snippets if you need anything else. They will get 
rotated out of elasticsearch within a few months and disappear, so hopefully we 
can get to the bottom of this before that.


{code}
3 node cluster. 
Broker 1 is controller.
Zookeeper GC pause meant that broker 3 lost connection. 
When it came back, broker 3 thought it was controller, but thought there were 
no alive brokers--see the empty set referenced in the logs below. This alone 
seems incorrect because if a broker is a controller, you'd think it would 
include itself in the set.


See the following in the logs:


[2017-03-17 21:32:15,812] ERROR Controller 3 epoch 134 initiated state change 
for partition [topic_name,626] from OfflinePartition to OnlinePartition failed 
(s
tate.change.logger)
kafka.common.NoReplicaOnlineException: No replica for partition 
[topic_name,626] is alive. Live brokers are: [Set()], Assigned replicas are: 
[List(1, 3)]
at 
kafka.controller.OfflinePartitionLeaderSelector.selectLeader(PartitionLeaderSelector.scala:75)
at 
kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:345)
at 
kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:205)
at 
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:120)
at 
kafka.controller.PartitionStateMachine$$anonfun$triggerOnlinePartitionStateChange$3.apply(PartitionStateMachine.scala:117)
at 
scala.collection.TraversableLike$WithFilter$$anonfun$foreach$1.apply(TraversableLike.scala:772)
at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(HashMap.scala:98)
at scala.collection.mutable.HashTable$class.foreachEntry(HashTable.scala:226)
at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:39)
at scala.collection.mutable.HashMap.foreach(HashMap.scala:98)
at 
scala.collection.TraversableLike$WithFilter.foreach(TraversableLike.scala:771)
at 
kafka.controller.PartitionStateMachine.triggerOnlinePartitionStateChange(PartitionStateMachine.scala:117)
at 
kafka.controller.PartitionStateMachine.startup(PartitionStateMachine.scala:70)
at 
kafka.controller.KafkaController.onControllerFailover(KafkaController.scala:335)
at 
kafka.controller.KafkaController$$anonfun$1.apply$mcV$sp(KafkaController.scala:166)
at kafka.server.ZookeeperLeaderElector.elect(ZookeeperLeaderElector.scala:84)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:146)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:231)
at 
kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:141)
at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:824)
at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)

Looking at the code + error message, the controller is unaware of active 
brokers. However, there are assigned replicas. We checked the log files under 
/data/kafka and they had m_times greater than the exception timestamp, plus our 
producers and consumers seemed to be working, so the cluster is successfully 
passing data around. The controller situation is just screwed up.


[2017-03-17 21:32:43,976] ERROR Controller 3 epoch 134 initiated state 
change for partition 

[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2017-04-06 Thread Dong Lin (JIRA)

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

Dong Lin commented on KAFKA-3042:
-

[~pengwei] Sorry, I didn't notice your message in Jan until Jeff replies to 
this ticket today. Sorry we still have this issue.

[~pengwei] [~jeffwidman] Do you have way to consistently reproduce this bug? If 
so, I will look into this and try to fix it.

> 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
>  Components: controller
>Affects Versions: 0.8.2.1
> Environment: jdk 1.7
> centos 6.4
>Reporter: Jiahongchao
>Assignee: Dong Lin
>  Labels: reliability
> Fix For: 0.11.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.15#6346)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2017-04-06 Thread Jeff Widman (JIRA)

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

Jeff Widman commented on KAFKA-3042:


We hit this on 0.10.0.1. 

Root cause was a really long zookeeper GC pause that caused the brokers to lose 
their connection. Producers / Consumers were working successfully as they'd 
established their connections to the brokers before the zk issue, so they kept 
happily working. But the broker logs were throwing these warnings about cached 
zkVersion not matching. And anything that required controller was broken, for 
example any newly created partitions didn't have leaders. 

I think this log message could be made more specific to show which znodes don't 
match.

I don't know if this error message is thrown whenever two znodes don't match, 
but in our case the ZK GC pause resulted in a race condition sequence where 
somehow the epoch of /controller znode did not match the partition controller 
epoch under /brokers znode. I'm not sure if it's possible to fix this, perhaps 
with the ZK multi-command where updates are transactional.

It took us a while to realize that was what the log message meant, so the log 
message could be made more specific to report exactly which znode paths don't 
match in zookeper.

For us, forcing a controller re-election by deleting the /controller znode 
immediately fixed the issue without having to restart brokers. 

> 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
>  Components: controller
>Affects Versions: 0.8.2.1
> Environment: jdk 1.7
> centos 6.4
>Reporter: Jiahongchao
>Assignee: Dong Lin
>  Labels: reliability
> Fix For: 0.11.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.15#6346)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2017-01-10 Thread Pengwei (JIRA)

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

Pengwei commented on KAFKA-3042:


[~lindong]  It seems I still encounter this issue after apply your patch in our 
test enviroment
The log is also logging the message "Cached zkVersion " and cannot recover

> 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
>Assignee: Dong Lin
>  Labels: reliability
> Fix For: 0.10.2.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-11-29 Thread James Cheng (JIRA)

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

James Cheng commented on KAFKA-3042:


If KAFKA-4443 in fact solves this issue, then I'm ecstatically happy. This 
issue has been biting us for almost a year. Thanks [~lindong]!

> 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
>Assignee: Dong Lin
>  Labels: reliability
> Fix For: 0.10.2.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-11-29 Thread Dong Lin (JIRA)

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

Dong Lin commented on KAFKA-3042:
-

We recently made a fix in Kafka that may have the addressed the problem 
mentioned this JIRA. See KAFKA-4443 for more information.

> 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
>  Labels: reliability
> Fix For: 0.10.2.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-10-29 Thread Martin Nowak (JIRA)

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

Martin Nowak commented on KAFKA-3042:
-

Out of curiosity, why does the controller data also use an epoch, instead of 
relying on ZooKeeper's data version and conditional updates of that?
While I understand the benefit of using epochs for partitions to reduce ZK load 
[¹], moving the concurrency handling of the single /controller znode into Kafka 
seems overly complex for no benefit.

[¹]: 
https://cwiki.apache.org/confluence/display/KAFKA/Kafka+replication+detailed+design+V2

> 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.2.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-10-29 Thread Martin Nowak (JIRA)

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

Martin Nowak commented on KAFKA-3042:
-

Yes, still happens with 0.10.0.1. Also just ran into this after the ZooKeeper 
cluster wasn't available for a while.
The fact that the ZooKeeper cluster recovers itself, but Kafka instances (2 out 
of 3 in my case) remain unavailable until a manual restart, makes it fairly 
unreliable.
Not sure why you'd loose messages though, your client just shouldn't be able to 
write.
Yes, ZooKeeper setups should be improved to be reliable, but that's a 
misleading angle, because session expiration can happen for several reasons and 
Kafka needs to properly deal with it.

> 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.2.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-09-27 Thread Andreas Schroeder (JIRA)

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

Andreas Schroeder commented on KAFKA-3042:
--

Since this issue wasn't updated for over a month, a short note that I ran into 
this issue as well: I tested with a local, 3-node Docker-based cluster when I 
repeatedly hunted down the controller broker and restarted him. Since I'm not 
yet that deep into Kafka and the codebase, I can only report from a user 
perspective the consequences I saw: 
* The failure to shrink the ISR affected partitions of all topics, including 
__consumer_offsets. 
* Two brokers ran repeatedly into this issue within 20 minutes.
* During that time, producers were unable to write data to the cluster
* Also during that time, I saw messages being lost even though I used a 
replication factor of 3, min insync replicas of 2 and acks=all.

> 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.1.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-08-03 Thread Guozhang Wang (JIRA)

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

Guozhang Wang commented on KAFKA-3042:
--

I agree that we should consider improving the controller logic as a general 
story, at the same time I feel it would not harm (we need to think through the 
corner cases for sure) to combine these two requests as from a debugging 
perspective.

> 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.1.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-08-02 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


[~onurkaraman], there are a couple of things.

1. Currently when a broker starts up, it expects the very first 
LeaderAndIsrRequest to contain all the partitions hosted on this broker. After 
that, we read the last checkpointed high watermark and start the high watermark 
checkpoint thread. If we combine UpdateMetadataRequest and LeaderAndIsrRequest, 
the very first request that a broker receives could be an UpdateMetadataRequest 
including partitions not hosted on this broker. Then, we may checkpoint high 
watermarks on incorrect partitions.

2. Currently, LeaderAndIsrRequest is used to inform replicas about the new 
leader and is only sent to brokers storing the partition. UpdateMetadataRequest 
is used for updating the metadata cache for the clients and is sent to every 
broker. Technically, they are for different things. So, using separate requests 
makes logical sense. We could use a single request to do both. Not sure if this 
makes it clearer or more confusing from a debugging perspective. In any case, 
there will be significant code changes to do this. I am not opposed to that. I 
just think that if we want to do that, we probably want to think through how to 
improve the controller logic holistically since there are other known pain 
points in the controller.


> 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.1.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-08-02 Thread Onur Karaman (JIRA)

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

Onur Karaman commented on KAFKA-3042:
-

I'm still trying to understand the downsides to getting rid of 
LeaderAndIsrRequest. Let's say we got rid of it. From a debugging standpoint, 
is there any information from the request log or mbeans that would no longer 
exist or be harder to figure out? If such a thing exists, maybe we can just 
augment the UpdateMetadataRequest or add an mbean to fill in that gap.

> 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.1.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-08-02 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


Thinking about this a bit more. An alternative approach is to extend 
LeaderAndIsrRequest to include all endpoints for live_leaders. The follower can 
then obtain the right host/port for the leader from 
LeaderAndIsrRequest.live_leaders, instead of the metadata cache. This approach 
is more general and won't be depending on the ordering of UpdateMetadataRequest 
and LeaderAndIsrRequest. Since this is a protocol change, we will need to do a 
KIP. With this change, LeaderAndIsrRequest and UpdateMetadataRequest will look 
almost identical. The later still has an extra field for rack per broker. We 
can look into combining the two requests in the future.

> 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.1.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-08-01 Thread Kane Kim (JIRA)

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

Kane Kim commented on KAFKA-3042:
-

We've found it's also being triggered by packet loss from broker to ZK node. 
Controller doesn't have to be killed.

> 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.1.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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-19 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


Currently, the main differences between LeaderAndIsrRequest and 
UpdateMetadataRequest are that (1) LeaderAndIsrRequest is only sent to brokers 
storing the partition while UpdateMetadataRequest is sent to every broker; (2) 
LeaderAndIsrRequest only includes the port used by interBrokerSecurityProtocol 
while UpdateMetadataRequest includes all ports. 

One of the original intentions of the separation is probably to save the amount 
of data communication, which may no longer be valid. It's probably simpler to 
combine LeaderAndIsrRequest and UpdateMetadataRequest into one request. 
However, we have to look at the code more carefully to understand the full 
implication. For example, in ReplicaManager, we have the logic that relies upon 
the very first LeaderAndIsrRequest to initialize the partition list for 
checkpointing the high watermarks.

To fix this particular issue, the simplest approach is to send 
UpdateMetadataRequest first during controller failover. It shouldn't do any 
harm since it just communicates the current state from ZK to all brokers.

It would be useful to fix a couple of other minor issues in this jira too.
1, Log liveBrokers in LeaderAndIsrRequest and UpdateMetadataRequest in 
state-change log.
2. LeaderAndIsrRequest has a controllerEpoch at the topic level and a 
controllerEpoch per partition. The former indicates the epoch of the controller 
that sends the LeaderAndIsrRequest. The latter indicates the epoch of the 
controller that last changed the leader/isr for that partition. When logging 
LeaderAndIsrRequest in state-change log in ReplicaManager, we mixed the two up 
in some of the cases. For example, in makeLeaders(), we log the top level 
controllerEpoch. In makeFollowers(), we log the controllerEpoch at the 
partition level. We should make the logging clearer and consistent.

> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-19 Thread Onur Karaman (JIRA)

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

Onur Karaman commented on KAFKA-3042:
-

It looks like LeaderAndIsrRequest structure is identical to 
UpdateMetadataRequest. Joel pointed me to KAFKA-901 for some background on this.

I feel like kafka's wire protocol would be simpler if the controller just sends 
out UpdateMetadataRequests to the rest of the cluster and we get rid of the 
concept of LeaderAndIsrRequest. Upon receiving UpdateMetadataRequest from a 
controller, a broker updates its caches and potentially does the work 
associated with becoming a leader or follower. Am I oversimplifying the 
situation?

> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-19 Thread Flavio Junqueira (JIRA)

[ 
https://issues.apache.org/jira/browse/KAFKA-3042?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-18 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


[~fpj], yes, I was confused by that initially too. The following is my finding 
after I dug a bit deeper. When a controller receives a broker change event, it 
will also log the live brokers after processing the event. So, in the 
controller log of broker 5, you will see the following matching entries.

{code}
[2016-04-09 00:41:35,442] INFO [BrokerChangeListener on Controller 5]: Broker 
change listener fired for path /brokers/ids with children 1,4,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
[2016-04-09 00:41:36,235] INFO [BrokerChangeListener on Controller 5]: Newly 
added brokers: , deleted brokers: 3, all live brokers: 1,4,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
{code}

However, after 

{code}
[2016-04-09 00:37:54,079] INFO [BrokerChangeListener on Controller 5]: Broker 
change listener fired for path /brokers/ids with children 1,2,3,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
{code}

you don't see the corresponding entry on "Newly added broker" afterward. The 
reason is that at 00:37:54, broker 5 has resigned as the controller and 
therefore ignored the broker change listener event. 

{code}
[2016-04-09 00:37:54,064] INFO [Controller 5]: Broker 5 resigned as the 
controller (kafka.controller.KafkaController)
{code}

Broker 5 became the controller again a bit later.

{code}
[2016-04-09 00:40:54,990] INFO [Controller 5]: Controller 5 incremented epoch 
to 416 (kafka.controller.KafkaController)
[2016-04-09 00:40:57,877] INFO [Controller 5]: Currently active brokers in the 
cluster: Set(1, 3, 4, 5) (kafka.controller.KafkaController)
{code}

As you can see, it initialized with 4 live brokers, including broker 4. It then 
sent the following LeaderAndIsrRequest to broker 1 after initialization.
{code}
[2016-04-09 00:40:58,089] TRACE Controller 5 epoch 416 sending become-follower 
LeaderAndIsr request (Leader:4,ISR:4,LeaderEpoch:364,ControllerEpoch:415) to 
broker 1 for partition [tec1.en2.frontend.syncPing,7] (state.change.logger)
{code}

The above LeaderAndIsrRequest likely includes broker 4 in the live broker list 
since there is no other indication that the live broker list has changed since 
the initialization. However, if broker 4 is not the metadata cache of broker 1 
(which seems to be the case), it can still lead to the following error.

{code}
[2016-04-09 00:40:58,144] ERROR Broker 1 received LeaderAndIsrRequest with 
correlation id 0 from controller 5 epoch 415 for partition 
[tec1.en2.frontend.syncPing,7] but cannot become follower since the new leader 
4 is unavailable. (state.change.logger)
{code}


> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-18 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

I checked the broker change listener output of broker 5:

{noformat}
[2016-04-09 00:37:54,079] INFO [BrokerChangeListener on Controller 5]: Broker 
change listener fired for path /brokers/ids with children 1,2,3,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
[2016-04-09 00:41:35,442] INFO [BrokerChangeListener on Controller 5]: Broker 
change listener fired for path /brokers/ids with children 1,4,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
{noformat}

These are the two events I could find that determine a time interval including 
the request event. The LeaderAndIsrRequest from broker 5 comes in at 
{{[2016-04-09 00:40:58,144]}}. This is in this comment:

https://issues.apache.org/jira/browse/KAFKA-3042?focusedCommentId=15236055=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15236055

> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-14 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


[~fpj], I don't think we log the liveBrokers in the leaderAndIsrRequest in the 
controller or the receiving broker. So, not sure how you confirmed that broker 
4 is not there in the leaderAndIsrRequest.

> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-14 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

[~junrao] In this comment:

https://issues.apache.org/jira/browse/KAFKA-3042?focusedCommentId=15236055=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15236055

 I showed that broker 5 is the one that sent the LeaderAndIsr request to broker 
1, and in here:

https://issues.apache.org/jira/browse/KAFKA-3042?focusedCommentId=15237383=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15237383

that broker 5 also didn't have broker 4 as a live broker when it sent the 
request to broker 1. It does sound right that the controller on failover should 
update the list of live brokers on other brokers before sending requests that 
make them followers or at least the problem should be transient in the sense 
that it could be corrected with a later message. However, it sounds like for 
the partition we are analyzing, there is this additional problem that 
controller 5 also didn't have broker 4 in its list of live brokers.

Interestingly, I also caught an instance of this:

{noformat}
[2016-04-09 00:37:54,111] DEBUG Sending MetadataRequest to 
Brokers:ArrayBuffer(2, 5)...
[2016-04-09 00:37:54,111] ERROR Haven't been able to send metadata update 
requests...
[2016-04-09 00:37:54,112] ERROR [Controller 5]: Forcing the controller to 
resign (kafka.controller.KafkaController)
{noformat}

I don't think this is related, but we have been wondering in another issue 
about the possible causes of batches in {{ControllerBrokerRequestBatch}} not 
being empty, and there are a few occurrences of it in these logs. This is 
happening, however, right after the controller resigns, so I'm guessing this is 
related to the controller shutting down:

{noformat}
[2016-04-09 00:37:54,064] INFO [Controller 5]: Broker 5 resigned as the 
controller (kafka.controller.KafkaController)
{noformat}

In any case, for this last issue, I'll create a jira to make sure that we have 
enough info to identify this issue when it happens. Currently, the exception is 
being propagated, but nowhere we are logging the cause.


> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-13 Thread Alexander Binzberger (JIRA)

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

Alexander Binzberger commented on KAFKA-3042:
-

In my case it was high load on the network.
Had this on the test machines/test cluster on openstack.
The physical network was at max load (peeks but sometimes for some more time) 
when I saw this.
I know this info is not very precise but might still help.

> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-13 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


[~ismael juma], we fixed a few issues related to soft failure post 0.8.2. 
However, there could be other issues that we don't know yet.

[~delbaeth], [~wushujames], even when we fix all the bugs related to soft 
failure, it would still be good to avoid it in the first place since it only 
adds overhead. A 6 zookeeper.session.timeout seems high though. Do you know 
what's causing the ZK session timeout? Is it related to GC or network?

> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-13 Thread Ismael Juma (JIRA)

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

Ismael Juma commented on KAFKA-3042:


[~junrao], some people have said that they have seen this issue in 0.8.2 too so 
that suggests that there may be 2 different problems?

> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-12 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


The issue seems to be the following. In 0.9.0, we changed the logic a bit in 
ReplicaManager.makeFollowers() to ensure that the new leader is in the 
liveBrokers of metadataCache. However, during a controller failover, the new 
controller first sends leaderAndIsr requests, followed by an UpdateMetaRequest. 
So, it is possible when a broker receives a leaderAndIsr request, the 
liveBrokers in metadataCache are stale and don't include the leader and 
therefore causes the becoming follower logic to error out. Indeed, from broker 
1's state-change log, the last UpdateMetaRequest before the error in becoming 
follower came from controller 1.

{code}
[2016-04-09 00:40:52,929] TRACE Broker 1 cached leader info 
(LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:330,ControllerEpoch:414),ReplicationFactor:3),AllReplicas:2,1,4)
 for partit
ion [tec1.usqe1.frontend.syncPing,1] in response to UpdateMetadata request sent 
by controller 1 epoch 414 with correlation id 877 (state.change.logger)
{code}

In controller 1's log, the last time it updated the live broker list is the 
following and it didn't include broker 4 in the live broker list.
{code}
[2016-04-09 00:39:33,005] INFO [BrokerChangeListener on Controller 1]: Newly 
added brokers: , deleted brokers: 2, all live brokers: 1,3,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
{code}

To fix this, we should probably send an UpdateMetadataRequest before any 
leaderAndIsrRequest during controller failover.

> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-12 Thread Robert Christ (JIRA)

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

Robert Christ commented on KAFKA-3042:
--

We turned on unclean leader election due to encountering KAFKA-3410.  We really 
want
to turn it back off but we need to get a bit of stability before we do.  We can 
live with the data
loss for the moment. 

We are experimenting with minimum ISR and how it behaves in various failure 
cases.
We expect to increase it for all of our replicated topics.  We overlooked it 
when we were starting
out and now we are trying to catch up.


> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-12 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

hey [~wushujames]

bq. you said that broker 3 failed to release leadership to broker 4 because 
broker 4 was offline

it is actually broker the one that failed to release leadership.

bq. What is the correct behavior for that scenario?

The behavior isn't incorrect in the following sense. We cannot completely 
prevent a single broker from being partitioned from the other replicas. If this 
broker is the leader before the partition, then it may remain in this state for 
some time. In the meanwhile, the other replicas may form a new ISR and make 
progress independently. But, very important, the partitioned broker won't be 
able to commit anything on its own, assuming that the minimum ISR is at least 
two.

In the scenario we are discussing, we don't have a network partition, but the 
behavior is equivalent: broker 1 will remain the leader until it is able to 
follow successfully. The part is bad is that broker 1 isn't partitioned away, 
it is talking to other controllers, and the broker should be brought back into 
a state that it can make progress with that partition and others that are 
equally stuck. The bottom line is that is safe, but we clearly want the broker 
up and making progress with those partitions.

Let me point out that from the logs, it looks like you have unclean leader 
election enabled because of this log message:

{noformat}
[2016-04-09 00:40:50,911] WARN [OfflinePartitionLeaderSelector]: No broker in 
ISR is alive for [tec1.en2.frontend.syncPing,7]. 
Elect leader 4 from live brokers 4. There's potential data loss. 
(kafka.controller.OfflinePartitionLeaderSelector)
{noformat} 

and no minimum ISR set:

{noformat}
[2016-04-09 00:56:53,009] WARN [Controller 5]: Cannot remove replica 1 from ISR 
of partition [tec1.en2.frontend.syncPing,7]
since it is not in the ISR. Leader = 4 ; ISR = List(4) 
{noformat}

Those options can cause some data loss.

> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-12 Thread James Cheng (JIRA)

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

James Cheng commented on KAFKA-3042:


Thanks [~fpj]. Do you need any additional info from us? I don't think we have 
any other logs, but let us know if you have any questions.

About your findings:
>From your comment at https://issues.apache.org/jira/browse/KAFKA-3042, you 
>said that broker 3 failed to release leadership to broker 4 because broker 4 
>was offline:
{noformat}
[2016-04-09 00:40:58,144] ERROR Broker 1 received LeaderAndIsrRequest with 
correlation id 0 from controller 5 epoch 415 for partition 
[tec1.en2.frontend.syncPing,7] but cannot become follower since the new leader 
4 is unavailable. (state.change.logger)
{noformat}
What is the correct behavior for that scenario? Should broker 3 continue 
leadership? Or should it give up leadership completely until a controller comes 
back and tells it who is the new leader? Does broker 3 send back a response (or 
error) to the controller saying that it was unable to accept that change?

What happens in this scenario?
1) Broker 1 is leader of a partition.
2) Controller sends a LeaderAndIsrRequest to brokers 1 and 2 and 3, saying that 
broker 4 is the new leader.
3) Brokers 2 and 3 receives the LeaderAndIsrRequest and accepts the change.
4) LeaderAndIsrRequest is delayed due to network latency enroute to broker 1.

During this delay, won't different brokers have different ideas of who the 
leader is? Broker 1 thinks it is leader. Brokers 2 3 4 5 think that broker 4 is 
the leader. Or did I miss something?




> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-12 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

I had a look at the zookeeper logs, and I couldn’t see anything unusual. There 
are session expirations, but it is expected that sessions expire.

Using the same topic-partition I used in my last comment, 
[tec1.en2.frontend.syncPing,7], I found that the reason seems to be that 
controller 5 is telling broker 1 that the partition leader is 4, but neither 5 
nor 1 think that broker 4 is up. Here are some relevant log lines from broker 5:

{noformat} 
[2016-04-09 00:37:54,079] INFO [BrokerChangeListener on Controller 5]: Broker 
change listener fired for path /brokers/ids with children 1,2,3,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
[2016-04-09 00:37:53,709] DEBUG [Partition state machine on Controller 5]: 
After leader election, leader cache is updated to Map…. 
[tec1.en2.frontend.syncPing,7] -> 
(Leader:2,ISR:2,LeaderEpoch:361,ControllerEpoch:410)
[2016-04-09 00:37:53,765] INFO [Partition state machine on Controller 5]: 
Started partition state machine with initial state -> Map… 
[tec1.en2.frontend.syncPing,7] -> OnlinePartition

[2016-04-09 00:40:58,415] DEBUG [Partition state machine on Controller 5]: 
After leader election, leader cache is updated to Map… 
[tec1.en2.frontend.syncPing,7] -> 
(Leader:4,ISR:4,LeaderEpoch:364,ControllerEpoch:415)

[2016-04-09 00:41:35,442] INFO [BrokerChangeListener on Controller 5]: Broker 
change listener fired for path /brokers/ids with children 1,4,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
{noformat}

Interestingly, broker 3 is the controller for epoch 415, see the last leader 
cache update, and this is the information that broker 1 receives from broker 5 
(see the previous comment). It looks like broker 5 ignored the fact that broker 
4 is down or at least not in its list of live brokers.  

Broker 3 seems to behave correctly with respect to the partition, here are some 
relevant log lines:

{noformat}
[2016-04-09 00:39:57,004] INFO [Controller 3]: Controller 3 incremented epoch 
to 415 (kafka.controller.KafkaController)

[2016-04-09 00:40:46,633] INFO [BrokerChangeListener on Controller 3]: Broker 
change listener fired for path /brokers/ids with children 3,4,5 
(kafka.controller.ReplicaStateMachine$BrokerChangeListener)
[2016-04-09 00:40:46,638] INFO [BrokerChangeListener on Controller 3]: Newly 
added brokers: 4, deleted brokers: , all live brokers: 3,4,5 
(kafka.controller.ReplicaStateMachine

[2016-04-09 00:40:50,911] DEBUG [OfflinePartitionLeaderSelector]: No broker in 
ISR is alive for [tec1.en2.frontend.syncPing,7]. Pick the leader from the alive 
assigned replicas: 4 (kafka.controller.OfflinePartitionLeaderSelector)
[2016-04-09 00:40:50,911] WARN [OfflinePartitionLeaderSelector]: No broker in 
ISR is alive for [tec1.en2.frontend.syncPing,7]. Elect leader 4 from live 
brokers 4. There's potential data loss. 
(kafka.controller.OfflinePartitionLeaderSelector)
[2016-04-09 00:40:50,911] INFO [OfflinePartitionLeaderSelector]: Selected new 
leader and ISR {"leader":4,"leader_epoch":364,"isr":[4]} for offline partition 
[tec1.en2.frontend.syncPing,7] (kafka.controller.OfflinePartitionLeaderSelector)

State-change log
[2016-04-09 00:40:50,909] TRACE Controller 3 epoch 415 started leader election 
for partition [tec1.en2.frontend.syncPing,7] (state.change.logger)
[2016-04-09 00:40:50,911] TRACE Controller 3 epoch 415 elected leader 4 for 
Offline partition [tec1.en2.frontend.syncPing,7] (state.change.logger)
[2016-04-09 00:40:50,930] TRACE Controller 3 epoch 415 changed partition 
[tec1.en2.frontend.syncPing,7] from OfflinePartition to OnlinePartition with 
leader 4 (state.change.logger)
{noformat}

To summarize, the problems seems to be that controller 5 tells broker 1 that 
the partition leader is an unavailable broker, and broker 1 fails to change the 
partition leader. As it fails to update the leader to broker 4, broker 1 
remains the leader, which causes it to keep trying to update the ISR and 
printing out the “Cached zkVersion…” messages. Broker 1 does not receive any 
controller update that enables it to correct the problem later on and 
consequently it is stuck with itself as partition leader incorrectly.  

> 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
> Attachments: controller.log, server.log.2016-03-23-01, 
> state-change.log
>
>
> sometimes one broker may repeatly log
> "Cached zkVersion 54 not 

[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-11 Thread Robert Christ (JIRA)

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

Robert Christ commented on KAFKA-3042:
--

The log on disk also stops and 00:39:46,246.  It does not resume until 
2016-04-09 01:00:54,529 in the next file.
The server log appears to continue right past this period until 00:59:15 which 
is when I did the controlled shutdown
starting at 00:58:54 (that is an approximate time based on my typing date).

James noticed that the log ended at the same period as when our zkCli.sh 
processes disconnected.



> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-11 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

Thanks for the logs. I still don’t have an answer, but I wanted to report some 
progress.

I tracked one of the partitions that I’ve chosen arbitrarily for which broker 1 
was complaining about the zk version: [tec1.en2.frontend.syncPing,7]. Here are 
a few things that I’ve observed:

Broker 1 is the leader of the partition:

{noformat}
[2016-04-09 00:40:52,883] TRACE Broker 1 cached leader info 
(LeaderAndIsrInfo:(Leader:1,ISR:1,LeaderEpoch:363,ControllerEpoch:414),ReplicationFactor:3),AllReplicas:1,2,4)
 for partition [tec1.en2.frontend.syncPing,7] in response to UpdateMetadata 
request sent by controller 1 epoch 414 with correlation id 876 
(state.change.logger)
{noformat}

but soon after it fails to release leadership to broker 4:

{noformat}
[2016-04-09 00:40:58,139] TRACE Broker 1 handling LeaderAndIsr request 
correlationId 0 from controller 5 epoch 416 starting the become-follower 
transition for partition [tec1.en2.frontend.syncPing,7] (state.change.logger)

[2016-04-09 00:40:58,144] ERROR Broker 1 received LeaderAndIsrRequest with 
correlation id 0 from controller 5 epoch 415 for partition 
[tec1.en2.frontend.syncPing,7] but cannot become follower since the new leader 
4 is unavailable. (state.change.logger)
{no format}

Now, a bit later in the log, the broker says that it is caching the leader info 
for the partition:

{noformat}
[2016-04-09 00:42:02,456] TRACE Broker 1 cached leader info 
(LeaderAndIsrInfo:(Leader:4,ISR:4,LeaderEpoch:364,ControllerEpoch:415),ReplicationFactor:3),AllReplicas:1,2,4)
 for partition [tec1.en2.frontend.syncPing,7] in response to UpdateMetadata 
request sent by controller 5 epoch 416 with correlation id 1473 
(state.change.logger)
{noformat}

but it keeps printing the “Cached zkVersion…” errors, which indicate that the 
broker still believes it is the leader of the partition, or at least the 
variable {{leaderReplicaIdOpt}} is set this way.

I also inspected other partitions, and the behavior doesn’t seem to be 
consistent. I’ve seen at least one partition in broker 2 for which the broker 
made the appropriate transition:

{noformat}
[2016-04-09 00:39:23,840] TRACE Broker 2 received LeaderAndIsr request 
(LeaderAndIsrInfo:(Leader:3,ISR:2,3,LeaderEpoch:305,ControllerEpoch:414),ReplicationFactor:3),AllReplicas:3,2,4)
 correlation id 535 from controller 1 epoch 414 for partition 
[tec1.ono_qe1.bodydata.recordings,20] (state.change.logger)
[2016-04-09 00:39:23,841] TRACE Broker 2 handling LeaderAndIsr request 
correlationId 535 from controller 1 epoch 414 starting the become-follower 
transition for partition [tec1.ono_qe1.bodydata.recordings,20] 
(state.change.logger)
[2016-04-09 00:39:23,841] TRACE Broker 2 stopped fetchers as part of 
become-follower request from controller 1 epoch 414 with correlation id 535 for 
partition [tec1.ono_qe1.bodydata.recordings,20] (state.change.logger)
[2016-04-09 00:39:23,856] TRACE Broker 2 truncated logs and checkpointed 
recovery boundaries for partition [tec1.ono_qe1.bodydata.recordings,20] as part 
of become-follower request with correlation id 535 from controller 1 epoch 414 
(state.change.logger)
[2016-04-09 00:39:23,856] TRACE Broker 2 started fetcher to new leader as part 
of become-follower request from controller 1 epoch 414 with correlation id 535 
for partition [tec1.ono_qe1.bodydata.recordings,20] (state.change.logger)
[2016-04-09 00:39:23,856] TRACE Broker 2 completed LeaderAndIsr request 
correlationId 535 from controller 1 epoch 414 for the become-follower 
transition for partition [tec1.ono_qe1.bodydata.recordings,20] 
(state.change.logger)
{noformat}

Actually, the state-change log of broker 2 seems to have a gap starting at 
{{[2016-04-09 00:39:46,246]}}. Is it when you’ve restarted the broker? 

> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-09 Thread Robert Christ (JIRA)

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

Robert Christ commented on KAFKA-3042:
--

Hi Flavio,

The logs can be found at:

https://s3-us-west-2.amazonaws.com/tivo-confluent/confluent.tar

Let me know if you have trouble accessing it.  It does include the zookeeper
logs.  We had just enabled the GC logs for zookeeper but it does cover the
period of the reproduction.

As for he zkCli.sh issue, I don't have a lot of information.  I was watching 
the kafka
logs and flipped back to my zkCli.sh session and it had disconnected and needed 
to
reconnect.  I do believe the session timed out and is probably the same root 
cause 
the causes our kafka broker sessions to timeout.




> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-09 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

[~delbaeth] thanks for all the information.

bq.  Broker 1 rolled correctly and rejoined and leadership rebalancing 
occurred. After broker 2 rolled and came back up it now has an inconsistent 
view of the metadata. It thinks there are only 300 topics and all the other 
brokers believe there are 700. Should we file this as a separate issue?

I'm not aware of this issue, so it does sound better to report it in a 
different jira and describe the problem in as much detail as possible. If you 
have logs for this problem, then please share.

bq. We have managed to reproduce the problem and have a snapshot of the logs. 
The tarball is about a gigabyte. What should I do with it?

If you have a web server that can host it, then perhaps you can upload it 
there. A dropbox/box/onedrive public folder that we can read from would also do 
it.

bq. my zkCli.sh session which I was using to watch the controller exited here 
so I was disconnected for a minute

This is odd. Could you describe in more detail what happened with the zkCli 
session? You said that it disconnected for a minute, but has the session 
expired? It must have expired, unless your session timeout was at least one 
minute, which according to your description, it wasn't. If you have them, 
please include the zk logs in the bundle.



  

> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-08 Thread Robert Christ (JIRA)

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

Robert Christ commented on KAFKA-3042:
--

Hello again,

We have managed to reproduce the problem and have a snapshot of the logs.
The tarball is about a gigabyte.  What should I do with it?

We spent a couple hours reproducing it and lots of exciting things seemed to 
happen.
I've added a basic timeline of our attempts to reproduce.  The pattern is 
essentially hard
kill the controller until bad things happen.  The first pass the controller 
only moved once
and we did not see any issues.  The second pass, the controller moved once and 
seemed
to work fine but 10 minutes later the controller moved twice more and we
see the "zkVersion" symptom but only briefly and the broker managed to recover.
The third time the controller bounced many times and we ended up seeing the 
problem
on broker 1 and broker 2 for a sustained period until we started restarting 
brokers
to try to recover the cluster.

The logs starting at 00:35 should be where we initiated the shutdown and were 
able
to reproduce the problem.

controller hard kill (5) 22:42:19
4 took controller
first try did not reproduce

controller hard kill (4) 22:48:10
2 took controller

1 took controller (22:59:06)
4 took controller (23:00:52)

1 showing cached zkVersion messages (23:00:13)
1 no longer showing cached zkVersion messages (23:01:10)

controller hard kill (4) 00:35:26
3 took controller 00:36:27
2 took controller 00:37:30
3 took controller 00:37:54
1 took controller 00:39:07

my zkCli.sh session which I was using to watch the controller exited here so I 
was disconnected for a minute

3 took controller 00:39:57
5 took controller 00:40:54

broker 1 has zkVersion problem
broker 2 has zkVersion problem



broker 1 controlled shutdown to fix (00:56:46)
broker 2 controlled shutdown to fix (00:58:54)

broker 2 appears to be shutdown but hasn't exited yet
hard kill broker 2 (01:00:38)

reverting back to 6ms timeout and restarting all the brokers (01:05:34)
broker 1 first (about a minute before 01:07:21)


> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-08 Thread Robert Christ (JIRA)

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

Robert Christ commented on KAFKA-3042:
--

Thanks Flavio and Jun for looking at this.

We had bumped our zookeeper.session.timeout to 6 which has allowed
us to avoid this problem.  In order to reproduce the problem again and get a
complete set of logs, we moved  the timeout back to the default and started
rolling the cluster.  After 2 brokers rolled we seem to have run in to a 
different
issue.  Broker 1 rolled correctly and rejoined and leadership rebalancing 
occurred.
After broker 2 rolled and came back up it now has an inconsistent view of the
metadata.  It thinks there are only 300 topics and all the other brokers believe
there are 700.

We are not sure if this is related in any way.  We did not see the controller 
change
(neither broker 1 or 2 were the controller) which is how we had reproduced this
problem in the past.

Should we file this as a separate issue?  We are going to go ahead and try to 
reproduce
the original symptoms.



> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-08 Thread Jun Rao (JIRA)

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

Jun Rao commented on KAFKA-3042:


[~delbaeth], [~wushujames], a few things.

1. Supposedly after step 5), controller 3 will send the latest ZK version for 
the ISR path to broker 2 through LeaderAndIsrRequest. That should stop the 
warning on "Cached zkVersion...". It seems somehow that didn't happen. Could 
you send the state-change log in broker 2 around that time? You want to include 
probably the log 5 mins before and 5 mins after the very first "Cached 
zkVersion...". Could you also do that for the controller log in controller 1 
and controller 3?

2. The controller log shows that controller 3 stopped at 01:05:23. Is broker 3 
still up at that time?

3. We have discovered a few issues due to ZK session expiration, not all of 
which have been fixed. So, in the short term, it would be good to avoid ZK 
session expiration in the first place. You mentioned this may be due to a 
network issue? How long did the network issue last? Another common cause of ZK 
session expiration is broker GC. Do you have the GC log on the session expired 
brokers?

> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-04-08 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

Here is what I've been able to find out so far based on the logs that 
[~wushujames] posted: 

# The "Cached zkVersion..." messages are printed in {{Partition.updateIsr}}. 
The {{updatedIsr}} in the logs seem to be called from 
{{Partition.maybeShrinkIsr}}.
# {{Partition.maybeShrinkIsr}} is called from 
{{ReplicaManager.maybeShrinkIsr}}, which is called periodically according to 
schedule call in {{ReplicaManager.startup}}. This is the main reason we see 
those messages periodically coming up.
# In {{Partition.maybeShrinkIsr}}, the ISR is only updated if the leader 
replica is the broker itself, which is determined by the variable 
{{leaderReplicaIdOpt}.

It looks like {{leaderReplicaIdOpt}} isn't being updated correctly, and it is 
possible that it is due to a race with either the controllers or the execution 
of {{LeaderAndIsr} requests.


> 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
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-02-23 Thread James Cheng (JIRA)

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

James Cheng commented on KAFKA-3042:


[~fpj], I have controller.log, state-change.log, and server.log for a broker 
that is experiencing this. When gzipped, the files are 100mb (in total for all 
3), (almost 1gig when uncompresssed) 

Is that too big to attach to JIRA? I can try slicing them smaller. I would need 
guidance on what parts are most important to you. In server.log, I see 'Cached 
zkVersion [45] not equal to that in zookeeper, skip updating ISR 
(kafka.cluster.Partition)' showing up as a specific time in the logs. I can 
send you the slices of all 3 logs around that timeframe.



> 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
>
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-02-23 Thread Flavio Junqueira (JIRA)

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

Flavio Junqueira commented on KAFKA-3042:
-

bq. 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.

But in this case, wouldn't the broker eventually recover? What I find 
problematic here is that the descriptions and comments in the related jiras 
mention the need of bouncing brokers, it shouldn't be necessary.

If anyone has state change logs to share for the period in which one of these 
incidents occurred, it'd be nice to see them.

> 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
>
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-01-15 Thread Alexander Binzberger (JIRA)

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

Alexander Binzberger commented on KAFKA-3042:
-

For me it looks like they are related or the same thing.
If someone from the Kafka team thinks different please undo the link.

> 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
>
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2016-01-15 Thread Alexander Binzberger (JIRA)

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

Alexander Binzberger commented on KAFKA-3042:
-

I think this is the same as: KAFKA-1382 and KAFKA-2729
Maybe also: KAFKA-1407

seen this on 0.9.0.0 college told me he might have seen it on 0.8.2

for 0.9 it happened after high network load (possible network outage) (and 
possible slow disk IO).
my test cluster is running on virtual machines on a open stack system. the 
virtual machines disk IO may also go over the network for read/write.
At the moment I see this once or twice a day.
The broker is just not recovering from that state.
If this happens I can not produce to at least some partitions.
This affects also the offset partitions what means I can not consume some 
things any more.
Taking down the whole cluster and restarting it resolves the issue but is no 
option for a production system.

The ISR state in zookeeper and topics.sh --describe show all partitions as 
perfectly in service. But the metadata over kafka protocol tells a different 
story (matching the kafka log).

http://pastebin.com/6ekA5w3a

> 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
>
> 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)


[jira] [Commented] (KAFKA-3042) updateIsr should stop after failed several times due to zkVersion issue

2015-12-30 Thread Petri Lehtinen (JIRA)

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

Petri Lehtinen commented on KAFKA-3042:
---

I've seen the same thing happen on version 0.9.0.0. It happened after Zookeeper 
recovered from a series of outages that took about 10 minutes.

> 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
>
> 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)