[ https://issues.apache.org/jira/browse/KAFKA-8151?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16799586#comment-16799586 ]
Joe Ammann commented on KAFKA-8151: ----------------------------------- For symptom 1 (failure of a broker to re-register with ZK) I have to freeze ZK plus 1 broker. This then normally shows itsself as follows * cluster is running ok * ZK and one broker get's frozen * partitions go to underreplicated on the remaining 2 brokers, as expected * failed broker and ZK comes back and reports that ZK session was expired {code:java} [2019-03-18 02:27:13,043] INFO [ZooKeeperClient] Session expired. (kafka.zookeeper.ZooKeeperClient){code} * the broker that failed does *not* re-appear under /brokers/ids in Zookeeper, but I can't also find specific error messages about failed tries to re-register. It almost looks as if it just doesn't try to re-register * some of the brokers that were ok report leader election problems {code:java} [2019-03-18 02:27:20,283] ERROR [Controller id=3 epoch=94562] Controller 3 epoch 94562 failed to change state for partition __consumer_offsets-4 from OnlinePartition to OnlinePartition (state.change.logger) kafka.common.StateChangeFailedException: Failed to elect leader for partition __consumer_offsets-4 under strategy PreferredReplicaPartitionLeaderElectionStrategy at kafka.controller.PartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:366) at kafka.controller.PartitionStateMachine$$anonfun$doElectLeaderForPartitions$3.apply(PartitionStateMachine.scala:364) at scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48) at kafka.controller.PartitionStateMachine.doElectLeaderForPartitions(PartitionStateMachine.scala:364) at kafka.controller.PartitionStateMachine.electLeaderForPartitions(PartitionStateMachine.scala:292) at kafka.controller.PartitionStateMachine.doHandleStateChanges(PartitionStateMachine.scala:210) at kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:133) at kafka.controller.KafkaController.kafka$controller$KafkaController$$onPreferredReplicaElection(KafkaController.scala:624) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:974) at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance$3.apply(KafkaController.scala:955) at scala.collection.immutable.Map$Map4.foreach(Map.scala:188) at kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerAutoLeaderRebalance(KafkaController.scala:955) at kafka.controller.KafkaController$AutoPreferredReplicaLeaderElection$.process(KafkaController.scala:986) at kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply$mcV$sp(ControllerEventManager.scala:89) at kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:89) at kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:89) at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31) at kafka.controller.ControllerEventManager$ControllerEventThread.doWork(ControllerEventManager.scala:88) at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82){code} * the failed/revived broker logs errors continuosly about expired session {code:java} [2019-03-18 02:28:34,493] ERROR Uncaught exception in scheduled task 'isr-expiration' (kafka.utils.KafkaScheduler) org.apache.zookeeper.KeeperException$SessionExpiredException: KeeperErrorCode = Session expired for /brokers/topics/__consumer_offsets/partitions/9/state at org.apache.zookeeper.KeeperException.create(KeeperException.java:130) at org.apache.zookeeper.KeeperException.create(KeeperException.java:54) at kafka.zookeeper.AsyncResponse.resultException(ZooKeeperClient.scala:539) at kafka.zk.KafkaZkClient.conditionalUpdatePath(KafkaZkClient.scala:717) at kafka.utils.ReplicationUtils$.updateLeaderAndIsr(ReplicationUtils.scala:33) at kafka.cluster.Partition.kafka$cluster$Partition$$updateIsr(Partition.scala:969) at kafka.cluster.Partition$$anonfun$2.apply$mcZ$sp(Partition.scala:642) at kafka.cluster.Partition$$anonfun$2.apply(Partition.scala:633) at kafka.cluster.Partition$$anonfun$2.apply(Partition.scala:633) at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:251) at kafka.utils.CoreUtils$.inWriteLock(CoreUtils.scala:259) at kafka.cluster.Partition.maybeShrinkIsr(Partition.scala:632) at kafka.server.ReplicaManager$$anonfun$kafka$server$ReplicaManager$$maybeShrinkIsr$2$$anonfun$apply$43.apply(ReplicaManager.scala:1349) at kafka.server.ReplicaManager$$anonfun$kafka$server$ReplicaManager$$maybeShrinkIsr$2$$anonfun$apply$43.apply(ReplicaManager.scala:1349) at scala.Option.foreach(Option.scala:257) at kafka.server.ReplicaManager$$anonfun$kafka$server$ReplicaManager$$maybeShrinkIsr$2.apply(ReplicaManager.scala:1349) at kafka.server.ReplicaManager$$anonfun$kafka$server$ReplicaManager$$maybeShrinkIsr$2.apply(ReplicaManager.scala:1348) at scala.collection.Iterator$class.foreach(Iterator.scala:891) at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at scala.collection.AbstractIterable.foreach(Iterable.scala:54) at kafka.server.ReplicaManager.kafka$server$ReplicaManager$$maybeShrinkIsr(ReplicaManager.scala:1348) at kafka.server.ReplicaManager$$anonfun$2.apply$mcV$sp(ReplicaManager.scala:323) at kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:114) at kafka.utils.CoreUtils$$anon$1.run(CoreUtils.scala:63) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java) at java.lang.Thread.run(Thread.java:745){code} > Broker hangs and lockups after Zookeeper outages > ------------------------------------------------ > > Key: KAFKA-8151 > URL: https://issues.apache.org/jira/browse/KAFKA-8151 > Project: Kafka > Issue Type: Bug > Components: controller, core, zkclient > Affects Versions: 2.1.1 > Reporter: Joe Ammann > Priority: Major > > We're running several clusters (mostly with 3 brokers) with 2.1.1, where we > see at least 3 different symptoms, all resulting on broker/controller lockups. > We are pretty sure that the triggering cause for all these symptoms are > temporary (for 3-5 minutes normally) of the Zookeeper cluster. The Linux VMs > where the ZK nodes run on regularly get stalled for a couple of minutes. The > ZK nodes always very quickly reunite and build a Quorum after the situation > clears, but the Kafka brokers (which run on then same Linux VMs) quite often > show problems after this procedure. > I've seen 3 different kinds of problems (this is why I put "reproduce" in > quotes, I can never predict what will happen) > # the brokers get their ZK sessions expired (obviously) and sometimes only 2 > of 3 re-register under /brokers/ids. The 3rd broker doesn't re-register for > some reason (that's the problem I originally described) > # the brokers all re-register and re-elect a new controller. But that new > controller does not fully work. For example it doesn't process partition > reassignment requests and or does not transfer partition leadership after I > kill a broker > # the previous controller gets "dead-locked" (it has 3-4 of the important > controller threads in a lock) and hence does not perform any of it's > controller duties. But it regards itsself still as the valid controller and > is accepted by the other brokers > I'll try to describe each one of the problems in more detail below, and hope > to be able to cleary separate them. > I'm able to provoke these problems in our DEV environment quite regularly > using the following procedure > * make sure all ZK nodes and Kafka brokers are stable and reacting normally > * freeze 2 out of 3 ZK nodes with {{kill -STOP}} for some minutes > * let the Kafka broker running, of course they will start complaining to be > unable to reach ZK > * thaw the processes with {{kill -CONT}} > * now all Kafka brokers get notified that their ZK session has expired, and > they start to reorganize the cluster > In about 20% of the tests, I'm able to produce one of the symptoms above. I > can not predict which one though. I'm varying this procedure sometimes by > also freezing one Kafka broker (most often the controller), but until now I > haven't been able to create a clear pattern or really force one specific > symptom > -- This message was sent by Atlassian JIRA (v7.6.3#76005)