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

Charly Molter edited comment on KAFKA-2729 at 9/6/16 3:38 PM:
--------------------------------------------------------------

Hi,

We had this issue on a test cluster running 0.10.0.0 so I took time to 
investigate some more.

We had a bunch of disconnections to Zookeeper and we had 2 changes of 
controller in a short time.

Broker 103 was controller with epoch 44
Broker 104 was controller with epoch 45

I looked at one specific partitions and found the following pattern:

101 was the broker which thought was leader but kept failing shrink the ISR 
with:
Partition [verifiable-test-topic,0] on broker 101: Shrinking ISR for partition 
[verifiable-test-topic,0] from 101,301,201 to 101,201
Partition [verifiable-test-topic,0] on broker 101: Cached zkVersion [185] not 
equal to that in zookeeper, skip updating ISR

Looking at ZK we have:
get /brokers/topics/verifiable-test-topic/partitions/0/state
{"controller_epoch":44,"leader":301,"version":1,"leader_epoch":96,"isr":[301]}

And metadata (to a random broker) is saying:
Topic: verifiable-test-topic    Partition: 0    Leader: 301     Replicas: 
101,201,301   Isr: 301

Digging in the logs here’s what we think happened:

1. 103 sends becomeFollower to 301 with epoch 44 and leaderEpoch 95
2. 104 sends becomeLeader to 101 with epoch 45 and leaderEpoch 95 (after update 
zk!)
3. 103 sends becomeLeader to 301 with epoch 44 and leaderEpoch 96 (after 
updating zk!)
4. 104 sends becomeFollower to 301 with epoch 45 and leaderEpoch 95

4) Is ignored by 301 as the leaderEpoch is older than the current one.

We are missing a request: 103 sends becomeFollower to 101 with epoch 44 and 
leaderEpoch 95

I believe this happened because when the controller steps down it empties its 
request queue so this request never left the controller: 
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/controller/ControllerChannelManager.scala#L53-L57

So we ended up in a case where 301 and 101 think they are both leaders. 
Obviously 101 wants to update the state in ZK to remove 301 as it’s not even 
fetching from 101.

Does this seem correct to you?

It seems impossible to avoid having no Controller overlap, which could make it 
quite hard to avoid having 2 leaders for a short time. Though there should be a 
way for this situation to get back to a good state.

I believe the impact of this would be:
- writes = -1 unavailability
- writes != -1 possible log divergence (I’m unsure about this).

Hope this helps. While I had to fix the cluster by bouncing a node I kept most 
of the logs so let me know if you need more info.


was (Author: cmolter):
Hi,

We had this issue on a test cluster running 0.10.0.0 so I took time to 
investigate some more.

We had a bunch of disconnections to Zookeeper and we had 2 changes of 
controller in a short time.

Broker 103 was leader with epoch 44
Broker 104 was leader with epoch 45

I looked at one specific partitions and found the following pattern:

101 was the broker which thought was leader but kept failing shrink the ISR 
with:
Partition [verifiable-test-topic,0] on broker 101: Shrinking ISR for partition 
[verifiable-test-topic,0] from 101,301,201 to 101,201
Partition [verifiable-test-topic,0] on broker 101: Cached zkVersion [185] not 
equal to that in zookeeper, skip updating ISR

Looking at ZK we have:
get /brokers/topics/verifiable-test-topic/partitions/0/state
{"controller_epoch":44,"leader":301,"version":1,"leader_epoch":96,"isr":[301]}

And metadata (to a random broker) is saying:
Topic: verifiable-test-topic    Partition: 0    Leader: 301     Replicas: 
101,201,301   Isr: 301

Digging in the logs here’s what we think happened:

1. 103 sends becomeFollower to 301 with epoch 44 and leaderEpoch 95
2. 104 sends becomeLeader to 101 with epoch 45 and leaderEpoch 95 (after update 
zk!)
3. 103 sends becomeLeader to 301 with epoch 44 and leaderEpoch 96 (after 
updating zk!)
4. 104 sends becomeFollower to 301 with epoch 45 and leaderEpoch 95

4) Is ignored by 301 as the leaderEpoch is older than the current one.

We are missing a request: 103 sends becomeFollower to 101 with epoch 44 and 
leaderEpoch 95

I believe this happened because when the controller steps down it empties its 
request queue so this request never left the controller: 
https://github.com/apache/kafka/blob/trunk/core/src/main/scala/kafka/controller/ControllerChannelManager.scala#L53-L57

So we ended up in a case where 301 and 101 think they are both leaders. 
Obviously 101 wants to update the state in ZK to remove 301 as it’s not even 
fetching from 101.

Does this seem correct to you?

It seems impossible to avoid having no Controller overlap, which could make it 
quite hard to avoid having 2 leaders for a short time. Though there should be a 
way for this situation to get back to a good state.

I believe the impact of this would be:
- writes = -1 unavailability
- writes != -1 possible log divergence (I’m unsure about this).

Hope this helps. While I had to fix the cluster by bouncing a node I kept most 
of the logs so let me know if you need more info.

> Cached zkVersion not equal to that in zookeeper, broker not recovering.
> -----------------------------------------------------------------------
>
>                 Key: KAFKA-2729
>                 URL: https://issues.apache.org/jira/browse/KAFKA-2729
>             Project: Kafka
>          Issue Type: Bug
>    Affects Versions: 0.8.2.1
>            Reporter: Danil Serdyuchenko
>
> After a small network wobble where zookeeper nodes couldn't reach each other, 
> we started seeing a large number of undereplicated partitions. The zookeeper 
> cluster recovered, however we continued to see a large number of 
> undereplicated partitions. Two brokers in the kafka cluster were showing this 
> in the logs:
> {code}
> [2015-10-27 11:36:00,888] INFO Partition 
> [__samza_checkpoint_event-creation_1,3] on broker 5: Shrinking ISR for 
> partition [__samza_checkpoint_event-creation_1,3] from 6,5 to 5 
> (kafka.cluster.Partition)
> [2015-10-27 11:36:00,891] INFO Partition 
> [__samza_checkpoint_event-creation_1,3] on broker 5: Cached zkVersion [66] 
> not equal to that in zookeeper, skip updating ISR (kafka.cluster.Partition)
> {code}
> For all of the topics on the effected brokers. Both brokers only recovered 
> after a restart. Our own investigation yielded nothing, I was hoping you 
> could shed some light on this issue. Possibly if it's related to: 
> https://issues.apache.org/jira/browse/KAFKA-1382 , however we're using 
> 0.8.2.1.



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

Reply via email to