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

Raj commented on KAFKA-2729:
----------------------------

Hi [~junrao] ,

This was just hit in our production as well although I was able to resolve it 
by only restarting the broker that reported errors as opposed to the controller 
or the whole cluster.

Kafka version : 2.3.1

I can confirm the events are identical to what [~l0co]  explained above. 
 * ZK session disconnected on broker 5
 * Replica Fetchers stopped on other brokers
 * ZK Connection re-established on broker 5 after a few seconds
 * Broker 5 came back online and started reporting the "Cached zkVersion[130] 
not equal to..." and shrunk ISRs to only itself

As it didn't recover automatically, I restarted the broker after 30 minutes and 
it then went back to normal.

I did see that the controller tried to send correct metadata to broker 5 but 
which was rejected due to epoch inconsistency.
{noformat}
ERROR [KafkaApi-5] Error when handling request: clientId=21, correlationId=2, 
api=UPDATE_METADATA, 
body={controller_id=21,controller_epoch=53,broker_epoch=223338313060,topic_states=[{topic-a,partition_states=[{partition=0,controller_epoch=53,leader=25,leader_epoch=70,isr=[25,17],zk_version=131,replicas=[5,25,17],offline_replicas=[]}...
...
java.lang.IllegalStateException: Epoch 223338313060 larger than current broker 
epoch 223338311791
        at kafka.server.KafkaApis.isBrokerEpochStale(KafkaApis.scala:2612)
        at kafka.server.KafkaApis.handleLeaderAndIsrRequest(KafkaApis.scala:194)
        at kafka.server.KafkaApis.handle(KafkaApis.scala:117)
        at kafka.server.KafkaRequestHandler.run(KafkaRequestHandler.scala:69)
        at java.base/java.lang.Thread.run(Thread.java:834)
...
...
...
[2021-07-29 11:07:30,210] INFO [Partition topic-a-0 broker=5] Cached zkVersion 
[130] not equal to that in zookeeper, skip updating ISR 
(kafka.cluster.Partition)
...

{noformat}
 

Preferred leader election error as seen on controller
{noformat}
[2021-07-29 11:11:57,432] ERROR [Controller id=21] Error completing preferred 
replica leader election for partition topic-a-0 
(kafka.controller.KafkaController)
kafka.common.StateChangeFailedException: Failed to elect leader for partition 
topic-a-0 under strategy PreferredReplicaPartitionLeaderElectionStrategy
        at 
kafka.controller.ZkPartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:381)
        at 
kafka.controller.ZkPartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:378)
        at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
        at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
        at 
kafka.controller.ZkPartitionStateMachine.doElectLeaderForPartitions(PartitionStateMachine.scala:378)
        at 
kafka.controller.ZkPartitionStateMachine.electLeaderForPartitions(PartitionStateMachine.scala:305)
        at 
kafka.controller.ZkPartitionStateMachine.doHandleStateChanges(PartitionStateMachine.scala:215)
        at 
kafka.controller.ZkPartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:145)
        at 
kafka.controller.KafkaController.kafka$controller$KafkaController$$onPreferredReplicaElection(KafkaController.scala:646)
        at 
kafka.controller.KafkaController$$anonfun$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:995)
        at 
kafka.controller.KafkaController$$anonfun$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:976)
        at 
scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:221)
        at 
scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:428)
        at 
kafka.controller.KafkaController.checkAndTriggerAutoLeaderRebalance(KafkaController.scala:976)
        at 
kafka.controller.KafkaController.processAutoPreferredReplicaLeaderElection(KafkaController.scala:1004)
        at kafka.controller.KafkaController.process(KafkaController.scala:1564)
        at kafka.controller.QueuedEvent.process(ControllerEventManager.scala:53)
        at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply$mcV$sp(ControllerEventManager.scala:137)
        at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:137)
        at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:137)
        at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
        at 
kafka.controller.ControllerEventManager$ControllerEventThread.doWork(ControllerEventManager.scala:136)
        at 
kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:89){noformat}
 

After the restart of broker-5, it was able to take back leadership of the 
desired partitions

 

Kindly let me know if anything else is needed.

 

> 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, 0.9.0.0, 0.10.0.0, 0.10.1.0, 0.11.0.0, 2.4.1
>            Reporter: Danil Serdyuchenko
>            Assignee: Onur Karaman
>            Priority: Critical
>             Fix For: 1.1.0
>
>
> 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
(v8.3.4#803005)

Reply via email to