You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@kafka.apache.org by "Patrick Lucas (JIRA)" <ji...@apache.org> on 2015/06/15 22:46:00 UTC

[jira] [Created] (KAFKA-2269) Partition imbalance does not converge to 0, even with auto.leader.rebalance.enable=true

Patrick Lucas created KAFKA-2269:
------------------------------------

             Summary: Partition imbalance does not converge to 0, even with auto.leader.rebalance.enable=true
                 Key: KAFKA-2269
                 URL: https://issues.apache.org/jira/browse/KAFKA-2269
             Project: Kafka
          Issue Type: Bug
          Components: controller
    Affects Versions: 0.8.2.1
            Reporter: Patrick Lucas
            Assignee: Neha Narkhede
         Attachments: graph.png

In the past four days I have replaced six brokers in a high-volume cluster. As the new broker comes up and replicates the data it is responsible for, the cluster-wide partition imbalance trends toward zero. But predictably, with around 19 partitions to go, the partition imbalance levels off and never reaches zero, even after all partitions have a full ISR. I waited as long as 24 hours to experiment with this, and in each case I had to manually run a preferred replica election to correct the imbalance.

The state-change log file on the controller does have a traceback for each partition contributing to the imbalance, as included below.

!graph.png!

{noformat}
[2015-06-15 09:43:03,034] ERROR Controller 172337636 epoch 113 encountered error while electing leader for partition [my.topic,1] due to: Preferred replica 172340284 for partition [my.topic,1] is either not alive or not in the isr. Current leader and ISR: [{"leader":172314088,"leader_epoch":923,"isr":[172314088,172322941]}]. (state.change.logger)
[2015-06-15 09:43:03,034] ERROR Controller 172337636 epoch 113 initiated state change for partition [my.topic,1] from OnlinePartition to OnlinePartition failed (state.change.logger)
kafka.common.StateChangeFailedException: encountered error while electing leader for partition [my.topic,1] due to: Preferred replica 172340284 for partition [my.topic,1] is either not alive or not in the isr. Current leader and ISR: [{"leader":172314088,"leader_epoch":923,"isr":[172314088,172322941]}].
	at kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:380)
	at kafka.controller.PartitionStateMachine.kafka$controller$PartitionStateMachine$$handleStateChange(PartitionStateMachine.scala:208)
	at kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:146)
	at kafka.controller.PartitionStateMachine$$anonfun$handleStateChanges$2.apply(PartitionStateMachine.scala:145)
	at scala.collection.immutable.Set$Set1.foreach(Set.scala:74)
	at kafka.controller.PartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:145)
	at kafka.controller.KafkaController.onPreferredReplicaElection(KafkaController.scala:631)
	at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17$$anonfun$apply$5.apply$mcV$sp(KafkaController.scala:1158)
	at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17$$anonfun$apply$5.apply(KafkaController.scala:1153)
	at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17$$anonfun$apply$5.apply(KafkaController.scala:1153)
	at kafka.utils.Utils$.inLock(Utils.scala:535)
	at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1150)
	at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4$$anonfun$apply$17.apply(KafkaController.scala:1148)
	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 kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1148)
	at kafka.controller.KafkaController$$anonfun$kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance$4.apply(KafkaController.scala:1127)
	at scala.collection.immutable.HashMap$HashMap1.foreach(HashMap.scala:224)
	at scala.collection.immutable.HashMap$HashTrieMap.foreach(HashMap.scala:403)
	at kafka.controller.KafkaController.kafka$controller$KafkaController$$checkAndTriggerPartitionRebalance(KafkaController.scala:1127)
	at kafka.controller.KafkaController$$anonfun$onControllerFailover$1.apply$mcV$sp(KafkaController.scala:326)
	at kafka.utils.KafkaScheduler$$anonfun$1.apply$mcV$sp(KafkaScheduler.scala:99)
	at kafka.utils.Utils$$anon$1.run(Utils.scala:54)
	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:617)
	at java.lang.Thread.run(Thread.java:745)
Caused by: kafka.common.StateChangeFailedException: Preferred replica 172340284 for partition [my.topic,1] is either not alive or not in the isr. Current leader and ISR: [{"leader":172314088,"leader_epoch":923,"isr":[172314088,172322941]}]
	at kafka.controller.PreferredReplicaPartitionLeaderSelector.selectLeader(PartitionLeaderSelector.scala:159)
	at kafka.controller.PartitionStateMachine.electLeaderForPartition(PartitionStateMachine.scala:357)
	... 32 more
{noformat}



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