Kang H Lee created KAFKA-8185:
---------------------------------

             Summary: Controller becomes stale and not able to failover the 
leadership for the partitions
                 Key: KAFKA-8185
                 URL: https://issues.apache.org/jira/browse/KAFKA-8185
             Project: Kafka
          Issue Type: Bug
          Components: controller
    Affects Versions: 1.1.1
            Reporter: Kang H Lee
         Attachments: broker12.zip, broker9.zip, zookeeper.zip

Description:

After broker 9 went offline, all partitions led by it went offline. The 
controller attempted to move leadership but ran into an exception while doing 
so:
{code:java}
// [2019-03-26 01:23:34,114] ERROR [PartitionStateMachine controllerId=12] 
Error while moving some partitions to OnlinePartition state 
(kafka.controller.PartitionStateMachine)
java.util.NoSuchElementException: key not found: me-test-1
at scala.collection.MapLike$class.default(MapLike.scala:228)
at scala.collection.AbstractMap.default(Map.scala:59)
at scala.collection.mutable.HashMap.apply(HashMap.scala:65)
at 
kafka.controller.PartitionStateMachine$$anonfun$14.apply(PartitionStateMachine.scala:202)
at 
kafka.controller.PartitionStateMachine$$anonfun$14.apply(PartitionStateMachine.scala:202)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at 
kafka.controller.PartitionStateMachine.initializeLeaderAndIsrForPartitions(PartitionStateMachine.scala:202)
at 
kafka.controller.PartitionStateMachine.doHandleStateChanges(PartitionStateMachine.scala:167)
at 
kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:116)
at 
kafka.controller.PartitionStateMachine.triggerOnlinePartitionStateChange(PartitionStateMachine.scala:106)
at 
kafka.controller.KafkaController.kafka$controller$KafkaController$$onReplicasBecomeOffline(KafkaController.scala:437)
at 
kafka.controller.KafkaController.kafka$controller$KafkaController$$onBrokerFailure(KafkaController.scala:405)
at 
kafka.controller.KafkaController$BrokerChange$.process(KafkaController.scala:1246)
at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply$mcV$sp(ControllerEventManager.scala:69)
at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:69)
at 
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:69)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
at 
kafka.controller.ControllerEventManager$ControllerEventThread.doWork(ControllerEventManager.scala:68)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:82)
{code}
The controller was unable to move leadership of partitions led by broker 9 as a 
result. It's worth noting that the controller ran into the same exception when 
the broker came back up online. The controller thinks `me-test-1` is a new 
partition and when attempting to transition it to an online partition, it is 
unable to retrieve its replica assignment from 
ControllerContext#partitionReplicaAssignment. I need to look through the code 
to figure out if there's a race condition or situations where we remove the 
partition from ControllerContext#partitionReplicaAssignment but might still 
leave it in PartitionStateMachine#partitionState.

They had to change the controller to recover from the offline status.

Sequential event:

* Broker 9 got restated in between : 2019-03-26 01:22:54,236 - 2019-03-26 
01:27:30,967: This was unclean shutdown.
* From 2019-03-26 01:27:30,967, broker 9 was rebuilding indexes. Broker 9 
wasn't able to process data at this moment.
* At 2019-03-26 01:29:36,741, broker 9 was starting to load replica.
* [2019-03-26 01:29:36,202] ERROR [KafkaApi-9] Number of alive brokers '0' does 
not meet the required replication factor '3' for the offsets topic (configured 
via 'offsets.topic.replication.factor'). This error can be ignored if the 
cluster is starting up and not all brokers are up yet. (kafka.server.KafkaApis)
* At 2019-03-26 01:29:37,270, broker 9 started report offline partitions.

 



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to