You are viewing a plain text version of this content. The canonical link for it is here.
Posted to users@kafka.apache.org by James Brown <jb...@easypost.com> on 2019/11/25 20:42:19 UTC

New topic gets stuck in bad state after Zookeeper error

We have an 8-node Kafka cluster that's been running for about four years
and is currently on Kafka 2.3.1. It's supported by a 3-node Zookeeper
cluster running ZK 3.5.5.

Last week, a topic got stuck in a weird state after it was auto-created:
kafka-topics.sh didn't show any leader at all (not the usual -1, just a
blank field). Output looked like the following:

Topic:reportgen.step_stats PartitionCount:1 ReplicationFactor:3 Configs:
Topic: reportgen.step_stats Partition: 0 Leader:  Replicas: 6,7,8 Isr:

Similarly, the JMX metrics for UnderReplicatedPartitions and
OfflinePartitionsCount both reported "0", despite this partition clearly
being un-available for writing.

I traced down the Kafka logs at the time of creation:

 [2019-11-22 00:26:01,040] INFO Creating topic reportgen.step_stats with
configuration {} and initial partition assignment Map(0 -> ArrayBuffer(6,
7, 8))
(kafka.zk.AdminZkClient)
 [2019-11-22 00:26:01,040] INFO Creating topic reportgen.step_stats with
configuration {} and initial partition assignment Map(0 -> ArrayBuffer(6,
7, 8))
(kafka.zk.AdminZkClient)
 [2019-11-22 00:26:01,106] INFO [KafkaApi-3] Auto creation of topic
reportgen.step_stats with 1 partitions and replication factor 3 is
successful (kafka.server.KafkaApis)
 [2019-11-22 00:26:01,106] INFO [KafkaApi-3] Auto creation of topic
reportgen.step_stats with 1 partitions and replication factor 3 is
successful (kafka.server.KafkaApis)
 [2019-11-22 00:26:01,162] INFO [Controller id=7] New topics:
[Set(reportgen.step_stats)], deleted topics: [Set()], new partition replica
assignment [Map(reportgen.step_stats-0 -> Vector(6, 7, 8))]
(kafka.controller.KafkaController)
 [2019-11-22 00:26:01,162] INFO [Controller id=7] New partition creation
callback for reportgen.step_stats-0 (kafka.controller.KafkaController)
 [2019-11-22 00:26:01,162] TRACE [Controller id=7 epoch=135] Changed
partition reportgen.step_stats-0 state from NonExistentPartition to
NewPartition with assigned replicas 6,7,8 (state.change.logger)
 [2019-11-22 00:26:01,162] TRACE [Controller id=7 epoch=135] Changed state
of replica 8 for partition reportgen.step_stats-0 from NonExistentReplica
to NewReplica (state.change.logger)
 [2019-11-22 00:26:01,162] TRACE [Controller id=7 epoch=135] Changed state
of replica 7 for partition reportgen.step_stats-0 from NonExistentReplica
to NewReplica (state.change.logger)
 [2019-11-22 00:26:01,162] TRACE [Controller id=7 epoch=135] Changed state
of replica 6 for partition reportgen.step_stats-0 from NonExistentReplica
to NewReplica (state.change.logger)
 [2019-11-22 00:26:01,427] ERROR [Controller id=7 epoch=135] Controller 7
epoch 135 failed to change state for partition reportgen.step_stats-0 from
NewPartition to NewPartition (state.change.logger)
 org.apache.zookeeper.KeeperException$SessionMovedException:
KeeperErrorCode = Session moved for /controller_epoch
        at
org.apache.zookeeper.KeeperException.create(KeeperException.java:134)
        at
org.apache.zookeeper.KeeperException.create(KeeperException.java:54)
        at
kafka.zk.KafkaZkClient$.kafka$zk$KafkaZkClient$$unwrapResponseWithControllerEpochCheck(KafkaZkClient.scala:1864)
        at
kafka.zk.KafkaZkClient$$anonfun$retryRequestsUntilConnected$2.apply(KafkaZkClient.scala:1650)
        at
kafka.zk.KafkaZkClient$$anonfun$retryRequestsUntilConnected$2.apply(KafkaZkClient.scala:1650)
        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.zk.KafkaZkClient.retryRequestsUntilConnected(KafkaZkClient.scala:1650)
        at
kafka.zk.KafkaZkClient.createTopicPartitions(KafkaZkClient.scala:1627)
        at
kafka.zk.KafkaZkClient.createTopicPartitionStatesRaw(KafkaZkClient.scala:214)
        at
kafka.controller.ZkPartitionStateMachine.initializeLeaderAndIsrForPartitions(PartitionStateMachine.scala:267)
        at
kafka.controller.ZkPartitionStateMachine.doHandleStateChanges(PartitionStateMachine.scala:207)
        at
kafka.controller.ZkPartitionStateMachine.handleStateChanges(PartitionStateMachine.scala:145)
        at
kafka.controller.KafkaController.onNewPartitionCreation(KafkaController.scala:490)
        at
kafka.controller.KafkaController.processTopicChange(KafkaController.scala:1319)
        at
kafka.controller.KafkaController.process(KafkaController.scala:1590)
        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)
 [2019-11-22 00:26:01,428] TRACE [Controller id=7 epoch=135] Changed state
of replica 8 for partition reportgen.step_stats-0 from NewReplica to
OnlineReplica (state.change.logger)
 [2019-11-22 00:26:01,428] TRACE [Controller id=7 epoch=135] Changed state
of replica 7 for partition reportgen.step_stats-0 from NewReplica to
OnlineReplica (state.change.logger)
 [2019-11-22 00:26:01,428] TRACE [Controller id=7 epoch=135] Changed state
of replica 6 for partition reportgen.step_stats-0 from NewReplica to
OnlineReplica (state.change.logger)

There's nothing at all in the Zookeeper logs from that time, and it's not
like the Kafka controller failed over. All other topics seemed fine. There
are also no subsequent logs about the new topic until I poked it today.

I bounced broker 6 and as soon as it came up, it assumed leadership of the
partition and everything started working fine.
Has anyone else seen this behavior before? The fact that a partition was
unavailable but the mbean showed 0 under-replicated and 0 un-available
topics is extremely concerning to me.

-- 
James Brown
Systems Engineer