Folks any input on this? Would help us a lot in case we get into this issue in future as right now only option is to delete the changelog table.
On Sat, Jun 10, 2017 at 12:19 AM, Sachin Mittal <sjmit...@gmail.com> wrote: > In course of our streaming application we discovered that it went into > hung state and upon further inspection we found that some of the partitions > had no leaders assigned. > > Here is the description of topic: > # bin/kafka-topics.sh --describe --zookeeper localhost:2181 --topic > new-part-advice-key-table-changelog > Topic:new-part-advice-key-table-changelog PartitionCount:12 > ReplicationFactor:3 Configs:retention.ms=1800000,cleanup.policy= > delete,compact > Topic: new-part-advice-key-table-changelog Partition: 0 > Leader: 6 Replicas: 6,4,5 Isr: 6,5,4 > Topic: new-part-advice-key-table-changelog Partition: 1 > Leader: 5 Replicas: 5,6,4 Isr: 5,4,6 > Topic: new-part-advice-key-table-changelog Partition: 2 > Leader: 4 Replicas: 4,5,6 Isr: 4,5,6 > Topic: new-part-advice-key-table-changelog Partition: 3 > Leader: 6 Replicas: 6,5,4 Isr: 5,6,4 > Topic: new-part-advice-key-table-changelog Partition: 4 > Leader: -1 Replicas: 5,4,6 Isr: 6 > Topic: new-part-advice-key-table-changelog Partition: 5 > Leader: 4 Replicas: 4,6,5 Isr: 6,4,5 > Topic: new-part-advice-key-table-changelog Partition: 6 > Leader: 6 Replicas: 6,4,5 Isr: 6,4,5 > Topic: new-part-advice-key-table-changelog Partition: 7 > Leader: 5 Replicas: 5,6,4 Isr: 5,4,6 > Topic: new-part-advice-key-table-changelog Partition: 8 > Leader: 4 Replicas: 4,5,6 Isr: 4,6,5 > Topic: new-part-advice-key-table-changelog Partition: 9 > Leader: 6 Replicas: 6,5,4 Isr: 5,6,4 > Topic: new-part-advice-key-table-changelog Partition: 10 > Leader: -1 Replicas: 5,4,6 Isr: 6 > Topic: new-part-advice-key-table-changelog Partition: 11 > Leader: 4 Replicas: 4,6,5 Isr: 4,6,5 > > Now to fix it we tried following commands > # ./bin/kafka-reassign-partitions.sh --reassignment-json-file > topicPartitionList.json --execute --zookeeper localhost:2181 > json file: > { > "partitions": [ > {"topic": "new-part-advice-key-table-changelog", "partition": 4, > "replicas": [5,4,6]}, > {"topic": "new-part-advice-key-table-changelog", "partition": 10, > "replicas": [5,4,6]} > ], > "version":1 > } > However this did not work and we saw following in the controller logs: > > [2017-06-09 19:59:33,499] DEBUG [PartitionsReassignedListener on 6]: > Partitions reassigned listener fired for path /admin/reassign_partitions. > Record partitions to be reassigned {"version":1,"partitions":[{"t > opic":"new-part-advice-key-table-changelog","partition":4," > replicas":[5,4,6]},{"topic":"new-part-advice-key-table- > changelog","partition":10,"replicas":[5,4,6]}]} > (kafka.controller.PartitionsReassignedListener) > [2017-06-09 19:59:33,506] ERROR [Controller 6]: Error completing > reassignment of partition [new-part-advice-key-table-changelog,4] ( > kafka.controller.KafkaController) > kafka.common.KafkaException: Partition [new-part-advice-key-table-changelog,4] > to be reassigned is already assigned to replicas 5,4,6. Ignoring request > for partition reassignment > at kafka.controller.KafkaController.initiateReassignReplicasFor > TopicPartition(KafkaController.scala:631) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4$$anonfun$apply$6.apply$mcV$sp(KafkaCont > roller.scala:1267) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4$$anonfun$apply$6.apply(KafkaController.scala:1261) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4$$anonfun$apply$6.apply(KafkaController.scala:1261) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4.apply(KafkaController.scala:1260) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4.apply(KafkaController.scala:1259) > at scala.collection.immutable.Map$Map2.foreach(Map.scala:130) > at kafka.controller.PartitionsReassignedListener.handleDataChan > ge(KafkaController.scala:1259) > at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:822) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > [2017-06-09 19:59:33,519] ERROR [Controller 6]: Error completing > reassignment of partition [new-part-advice-key-table-changelog,10] > (kafka.controller.KafkaController) > kafka.common.KafkaException: Partition > [new-part-advice-key-table-changelog,10] > to be reassigned is already assigned to replicas 5,4,6. Ignoring request > for partition reassignment > at kafka.controller.KafkaController.initiateReassignReplicasFor > TopicPartition(KafkaController.scala:631) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4$$anonfun$apply$6.apply$mcV$sp(KafkaCont > roller.scala:1267) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4$$anonfun$apply$6.apply(KafkaController.scala:1261) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4$$anonfun$apply$6.apply(KafkaController.scala:1261) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4.apply(KafkaController.scala:1260) > at kafka.controller.PartitionsReassignedListener$$anonfun$ > handleDataChange$4.apply(KafkaController.scala:1259) > at scala.collection.immutable.Map$Map2.foreach(Map.scala:130) > at kafka.controller.PartitionsReassignedListener.handleDataChan > ge(KafkaController.scala:1259) > at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:822) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > [2017-06-09 19:59:36,236] TRACE [Controller 6]: checking need to trigger > partition rebalance (kafka.controller.KafkaController) > ... > [2017-06-09 19:59:36,240] DEBUG [Controller 6]: topics not in preferred > replica Map([__consumer_offsets,36] -> List(5, 4, 6), > [new-part-advice-key-table-changelog,4] -> List(5, 4, 6), > [new-part-advice-key-table-changelog,10] -> List(5, 4, 6)) > (kafka.controller.KafkaController) > [2017-06-09 19:59:36,240] TRACE [Controller 6]: leader imbalance ratio for > broker 5 is 0.115385 (kafka.controller.KafkaController) > [2017-06-09 19:59:36,241] INFO [Controller 6]: Starting preferred replica > leader election for partitions [__consumer_offsets,36] (kafka.controller. > KafkaController) > [2017-06-09 19:59:36,241] INFO [Partition state machine on Controller 6]: > Invoking state change to OnlinePartition for partitions > [__consumer_offsets,36] (kafka.controller.PartitionStateMachine) > [2017-06-09 19:59:36,254] INFO [PreferredReplicaPartitionLeaderSelector]: > Current leader -1 for partition [__consumer_offsets,36] is not the > preferred replica. Trigerring preferred replica leader election > (kafka.controller.PreferredReplicaPartitionLeaderSelector) > [2017-06-09 19:59:36,262] WARN [Controller 6]: Partition > [__consumer_offsets,36] failed to complete preferred replica leader > election. Leader is -1 (kafka.controller.KafkaController) > [2017-06-09 19:59:36,263] INFO [Controller 6]: Starting preferred replica > leader election for partitions [new-part-advice-key-table-changelog,4] > (kafka.controller.KafkaController) > [2017-06-09 19:59:36,264] INFO [Partition state machine on Controller 6]: > Invoking state change to OnlinePartition for partitions > [new-part-advice-key-table-changelog,4] (kafka.controller.PartitionSta > teMachine) > [2017-06-09 19:59:36,299] INFO [PreferredReplicaPartitionLeaderSelector]: > Current leader -1 for partition [new-part-advice-key-table-changelog,4] > is not the preferred replica. Trigerring preferred replica leader election > (kafka.controller.PreferredReplicaPartitionLeaderSelector) > [2017-06-09 19:59:36,300] WARN [Controller 6]: Partition > [new-part-advice-key-table-changelog,4] failed to complete preferred > replica leader election. Leader is -1 (kafka.controller.KafkaController) > [2017-06-09 19:59:36,300] INFO [Controller 6]: Starting preferred replica > leader election for partitions [new-part-advice-key-table-changelog,10] > (kafka.controller.KafkaController) > [2017-06-09 19:59:36,301] INFO [Partition state machine on Controller 6]: > Invoking state change to OnlinePartition for partitions > [new-part-advice-key-table-changelog,10] (kafka.controller.PartitionSta > teMachine) > [2017-06-09 19:59:36,305] INFO [PreferredReplicaPartitionLeaderSelector]: > Current leader -1 for partition [new-part-advice-key-table-changelog,10] > is not the preferred replica. Trigerring preferred replica leader election > (kafka.controller.PreferredReplicaPartitionLeaderSelector) > [2017-06-09 19:59:36,307] WARN [Controller 6]: Partition > [new-part-advice-key-table-changelog,10] failed to complete preferred > replica leader election. Leader is -1 (kafka.controller.KafkaController) > [2017-06-09 19:59:36,308] DEBUG [Controller 6]: topics not in preferred > replica Map([new-advice-key-table-changelog,0] -> List(4, 5, 6), [ad > vice-stream,11] -> List(4, 5, 6)) (kafka.controller.KafkaController) > [2017-06-09 19:59:36,308] TRACE [Controller 6]: leader imbalance ratio for > broker 4 is 0.080000 (kafka.controller.KafkaController) > [2017-06-09 19:59:36,308] DEBUG [Controller 6]: topics not in preferred > replica Map() (kafka.controller.KafkaController) > [2017-06-09 19:59:36,308] TRACE [Controller 6]: leader imbalance ratio for > broker 6 is 0.000000 (kafka.controller.KafkaController) > > > We also tried this command and this also did not work > > # ./bin/kafka-preferred-replica-election.sh --zookeeper localhost:2181 > --path-to-json-file topicPartitionList.json > {"partitions":[ > {"topic": "new-part-advice-key-table-changelog", "partition": 4}, > {"topic": "new-part-advice-key-table-changelog", "partition": 10} > ] > } > > Here are the controller logs for the same: > > [2017-06-09 20:07:19,568] DEBUG [PreferredReplicaElectionListener on 6]: > Preferred replica election listener fired for path > /admin/preferred_replica_election. > Record partitions to undergo preferred replica election > {"version":1,"partitions":[{"topic":"new-part-advice-key-tab > le-changelog","partition":4},{"topic":"new-part-advice-key-t > able-changelog","partition":10}]} (kafka.controller.PreferredRep > licaElectionListener) > [2017-06-09 20:07:19,571] INFO [Controller 6]: Starting preferred replica > leader election for partitions [new-part-advice-key-table-cha > ngelog,4],[new-part-advice-key-table-changelog,10] (kafka.controller. > KafkaController) > [2017-06-09 20:07:19,572] INFO [Partition state machine on Controller 6]: > Invoking state change to OnlinePartition for partitions > [new-part-advice-key-table-changelog,4],[new-part-advice-key-table-changelog,10] > (kafka.controller.PartitionStateMachine) > [2017-06-09 20:07:19,577] INFO [PreferredReplicaPartitionLeaderSelector]: > Current leader -1 for partition [new-part-advice-key-table-changelog,4] > is not the preferred replica. Trigerring preferred replica leader election > (kafka.controller.PreferredReplicaPartitionLeaderSelector) > [2017-06-09 20:07:19,585] INFO [PreferredReplicaPartitionLeaderSelector]: > Current leader -1 for partition [new-part-advice-key-table-changelog,10] > is not the preferred replica. Trigerring preferred replica leader election > (kafka.controller.PreferredReplicaPartitionLeaderSelector) > [2017-06-09 20:07:19,587] WARN [Controller 6]: Partition > [new-part-advice-key-table-changelog,4] failed to complete preferred > replica leader election. Leader is -1 (kafka.controller.KafkaController) > [2017-06-09 20:07:19,588] WARN [Controller 6]: Partition > [new-part-advice-key-table-changelog,10] failed to complete preferred > replica leade > r election. Leader is -1 (kafka.controller.KafkaController) > > So any idea what is the cause and how can we fix this. We tried the two > commands based in kafka docs and tools wiki page and not sure why these did > not work. > > Any idea how can this be fixed. > > Thanks > Sachin > >