Details
-
Bug
-
Status: Open
-
Major
-
Resolution: Unresolved
-
3.1.0, 3.0.0, 3.0.1, 3.2.0, 3.1.1, 3.3.0, 3.0.2, 3.1.2, 3.2.1
-
None
Description
The controller runs into an IllegalStateException when reacting to changes in broker membership status if there are topics that are pending deletion.
How to reproduce:
- Setup cluster with 3 brokers
- Create a topic with a partition being led by each broker and produce some data
- Kill one of the brokers that is not the controller, and keep that broker down
- Delete the topic
- Restart the other broker that is not the controller
Logs and stacktrace:
[2022-05-16 11:53:25,482] ERROR [Controller id=1 epoch=1] Controller 1 epoch 1 initiated state change of replica 3 for partition test-topic-2 from ReplicaDeletionSuccessful to ReplicaDeletionIneligible failed (state.change.logger)
java.lang.IllegalStateException: Replica [Topic=test-topic,Partition=2,Replica=3] should be in the OfflineReplica,ReplicaDeletionStarted states before moving to ReplicaDeletionIneligible state. Instead it is in ReplicaDeletionSuccessful state
at kafka.controller.ZkReplicaStateMachine.logInvalidTransition(ReplicaStateMachine.scala:442)
at kafka.controller.ZkReplicaStateMachine.$anonfun$doHandleStateChanges$2(ReplicaStateMachine.scala:164)
at kafka.controller.ZkReplicaStateMachine.$anonfun$doHandleStateChanges$2$adapted(ReplicaStateMachine.scala:164)
at scala.collection.immutable.List.foreach(List.scala:333)
at kafka.controller.ZkReplicaStateMachine.doHandleStateChanges(ReplicaStateMachine.scala:164)
at kafka.controller.ZkReplicaStateMachine.$anonfun$handleStateChanges$2(ReplicaStateMachine.scala:112)
at kafka.controller.ZkReplicaStateMachine.$anonfun$handleStateChanges$2$adapted(ReplicaStateMachine.scala:111)
at kafka.utils.Implicits$MapExtensionMethods$.$anonfun$forKeyValue$1(Implicits.scala:62)
at scala.collection.immutable.HashMap.foreachEntry(HashMap.scala:1092)
at kafka.controller.ZkReplicaStateMachine.handleStateChanges(ReplicaStateMachine.scala:111)
at kafka.controller.TopicDeletionManager.failReplicaDeletion(TopicDeletionManager.scala:157)
at kafka.controller.KafkaController.onReplicasBecomeOffline(KafkaController.scala:638)
at kafka.controller.KafkaController.onBrokerFailure(KafkaController.scala:599)
at kafka.controller.KafkaController.processBrokerChange(KafkaController.scala:1623)
at kafka.controller.KafkaController.process(KafkaController.scala:2534)
at kafka.controller.QueuedEvent.process(ControllerEventManager.scala:52)
at kafka.controller.ControllerEventManager$ControllerEventThread.process$1(ControllerEventManager.scala:130)
--
[2022-05-16 11:53:40,726] ERROR [Controller id=1 epoch=1] Controller 1 epoch 1 initiated state change of replica 3 for partition test-topic-2 from ReplicaDeletionSuccessful to OnlineReplica failed (state.change.logger)
java.lang.IllegalStateException: Replica [Topic=test-topic,Partition=2,Replica=3] should be in the NewReplica,OnlineReplica,OfflineReplica,ReplicaDeletionIneligible states before moving to OnlineReplica state. Instead it is in ReplicaDeletionSuccessful state
at kafka.controller.ZkReplicaStateMachine.logInvalidTransition(ReplicaStateMachine.scala:442)
at kafka.controller.ZkReplicaStateMachine.$anonfun$doHandleStateChanges$2(ReplicaStateMachine.scala:164)
at kafka.controller.ZkReplicaStateMachine.$anonfun$doHandleStateChanges$2$adapted(ReplicaStateMachine.scala:164)
at scala.collection.immutable.List.foreach(List.scala:333)
at kafka.controller.ZkReplicaStateMachine.doHandleStateChanges(ReplicaStateMachine.scala:164)
at kafka.controller.ZkReplicaStateMachine.$anonfun$handleStateChanges$2(ReplicaStateMachine.scala:112)
at kafka.controller.ZkReplicaStateMachine.$anonfun$handleStateChanges$2$adapted(ReplicaStateMachine.scala:111)
at kafka.utils.Implicits$MapExtensionMethods$.$anonfun$forKeyValue$1(Implicits.scala:62)
at scala.collection.immutable.HashMap.foreachEntry(HashMap.scala:1092)
at kafka.controller.ZkReplicaStateMachine.handleStateChanges(ReplicaStateMachine.scala:111)
at kafka.controller.KafkaController.onBrokerStartup(KafkaController.scala:543)
at kafka.controller.KafkaController.processBrokerChange(KafkaController.scala:1607)
at kafka.controller.KafkaController.process(KafkaController.scala:2534)
at kafka.controller.QueuedEvent.process(ControllerEventManager.scala:52)
at kafka.controller.ControllerEventManager$ControllerEventThread.process$1(ControllerEventManager.scala:130)
at kafka.controller.ControllerEventManager$ControllerEventThread.$anonfun$doWork$1(ControllerEventManager.scala:133)
at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18)