Adam Kotwasinski created KAFKA-6438:
---------------------------------------
Summary: NSEE while concurrently creating and deleting a topic
Key: KAFKA-6438
URL: https://issues.apache.org/jira/browse/KAFKA-6438
Project: Kafka
Issue Type: Bug
Components: controller
Affects Versions: 1.0.0
Environment: kafka_2.11-1.0.0.jar
OpenJDK Runtime Environment (build 1.8.0_102-b14), OpenJDK 64-Bit Server VM
(build 25.102-b14, mixed mode)
CentOS Linux release 7.3.1611 (Core)
Reporter: Adam Kotwasinski
It appears that deleting a topic and creating it at the same time can cause
NSEE, what later results in a forced controller shutdown.
Most probably topics are being created because consumers/producers are still
active (yes, this means the deletion is happening blindly).
The main problem here (for me) is the controller switch, the data loss and
following unclean election is acceptable (as we admit to deleting blindly).
Environment description:
20 kafka brokers
80k partitions (20k topics 4partitions each)
3 node ZK
Incident:
{code:java}
[2018-01-09 11:19:05,912] INFO [Topic Deletion Manager 6], Partition deletion
callback for mytopic-2,mytopic-0,mytopic-1,mytopic-3
(kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:06,237] INFO [Controller id=6] New leader and ISR for
partition mytopic-0 is {"leader":-1,"leader_epoch":1,"isr":[]}
(kafka.controller.KafkaController)
[2018-01-09 11:19:06,412] INFO [Topic Deletion Manager 6], Deletion for
replicas 12,9,10,11 for partition mytopic-3,mytopic-0,mytopic-1,mytopic-2 of
topic mytopic in progress (kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:07,218] INFO [Topic Deletion Manager 6], Deletion for
replicas 12,9,10,11 for partition mytopic-3,mytopic-0,mytopic-1,mytopic-2 of
topic mytopic in progress (kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:07,304] INFO [Topic Deletion Manager 6], Deletion for
replicas 12,9,10,11 for partition mytopic-3,mytopic-0,mytopic-1,mytopic-2 of
topic mytopic in progress (kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:07,383] INFO [Topic Deletion Manager 6], Deletion for
replicas 12,9,10,11 for partition mytopic-3,mytopic-0,mytopic-1,mytopic-2 of
topic mytopic in progress (kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:07,510] INFO [Topic Deletion Manager 6], Deletion for
replicas 12,9,10,11 for partition mytopic-3,mytopic-0,mytopic-1,mytopic-2 of
topic mytopic in progress (kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:07,661] INFO [Topic Deletion Manager 6], Deletion for
replicas 12,9,10,11 for partition mytopic-3,mytopic-0,mytopic-1,mytopic-2 of
topic mytopic in progress (kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:07,728] INFO [Topic Deletion Manager 6], Deletion for
replicas 9,10,11 for partition mytopic-0,mytopic-1,mytopic-2 of topic mytopic
in progress (kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:07,924] INFO [PartitionStateMachine controllerId=6] Invoking
state change to OfflinePartition for partitions
mytopic-2,mytopic-0,mytopic-1,mytopic-3 (kafka.controller.PartitionStateMachine)
[2018-01-09 11:19:07,924] INFO [PartitionStateMachine controllerId=6] Invoking
state change to NonExistentPartition for partitions
mytopic-2,mytopic-0,mytopic-1,mytopic-3 (kafka.controller.PartitionStateMachine)
[2018-01-09 11:19:08,592] INFO [Controller id=6] New topics: [Set(mytopic,
other, querytickle_WD2-SALES1_espgms0202v29)], deleted topics: [Set()], new
partition replica assignment [Map(other-0 -> Vector(8), mytopic-2 -> Vector(6),
mytopic-0 -> Vector(4), other-2 -> Vector(10), mytopic-1 -> Vector(5),
mytopic-3 -> Vector(7), other-1 -> Vector(9), other-3 -> Vector(11))]
(kafka.controller.KafkaController)
[2018-01-09 11:19:08,593] INFO [Controller id=6] New topic creation callback
for other-0,mytopic-2,mytopic-0,other-2,mytopic-1,mytopic-3,other-1,other-3
(kafka.controller.KafkaController)
[2018-01-09 11:19:08,596] INFO [Controller id=6] New partition creation
callback for
other-0,mytopic-2,mytopic-0,other-2,mytopic-1,mytopic-3,other-1,other-3
(kafka.controller.KafkaController)
[2018-01-09 11:19:08,596] INFO [PartitionStateMachine controllerId=6] Invoking
state change to NewPartition for partitions
other-0,mytopic-2,mytopic-0,other-2,mytopic-1,mytopic-3,other-1,other-3
(kafka.controller.PartitionStateMachine)
[2018-01-09 11:19:08,642] INFO [PartitionStateMachine controllerId=6] Invoking
state change to OnlinePartition for partitions
other-0,mytopic-2,mytopic-0,other-2,mytopic-1,mytopic-3,other-1,other-3
(kafka.controller.PartitionStateMachine)
[2018-01-09 11:19:08,828] INFO [Topic Deletion Manager 6], Partition deletion
callback for mytopic-2,mytopic-0,mytopic-1,mytopic-3
(kafka.controller.TopicDeletionManager)
[2018-01-09 11:19:09,127] INFO [Controller id=6] New leader and ISR for
partition mytopic-0 is {"leader":-1,"leader_epoch":1,"isr":[]}
(kafka.controller.KafkaController)
[2018-01-09 11:19:09,607] ERROR [controller-event-thread]: Error processing
event TopicDeletion(Set(mytopic, other)) (kafka.controller.Contr
ollerEventManager$ControllerEventThread)
java.util.NoSuchElementException: key not found: mytopic-0
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.ControllerBrokerRequestBatch.kafka$controller$ControllerBrokerRequestBatch$$updateMetadataRequestPartitionInfo$1(ControllerChannelManager.scala:358)
at
kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$3.apply(ControllerChannelManager.scala:394)
at
kafka.controller.ControllerBrokerRequestBatch$$anonfun$addUpdateMetadataRequestForBrokers$3.apply(ControllerChannelManager.scala:394)
at scala.collection.mutable.HashSet.foreach(HashSet.scala:78)
at
kafka.controller.ControllerBrokerRequestBatch.addUpdateMetadataRequestForBrokers(ControllerChannelManager.scala:394)
at
kafka.controller.KafkaController.sendUpdateMetadataRequest(KafkaController.scala:1039)
at
kafka.controller.TopicDeletionManager.kafka$controller$TopicDeletionManager$$onTopicDeletion(TopicDeletionManager.scala:266)
at
kafka.controller.TopicDeletionManager$$anonfun$resumeDeletions$2.apply(TopicDeletionManager.scala:361)
at
kafka.controller.TopicDeletionManager$$anonfun$resumeDeletions$2.apply(TopicDeletionManager.scala:333)
at scala.collection.immutable.Set$Set3.foreach(Set.scala:163)
at
kafka.controller.TopicDeletionManager.resumeDeletions(TopicDeletionManager.scala:333)
at
kafka.controller.TopicDeletionManager.enqueueTopicsForDeletion(TopicDeletionManager.scala:113)
at
kafka.controller.KafkaController$TopicDeletion.process(KafkaController.scala:1299)
at
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply$mcV$sp(ControllerEventManager.scala:53)
at
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:53)
at
kafka.controller.ControllerEventManager$ControllerEventThread$$anonfun$doWork$1.apply(ControllerEventManager.scala:53)
at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:31)
at
kafka.controller.ControllerEventManager$ControllerEventThread.doWork(ControllerEventManager.scala:52)
at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64)
[2018-01-09 11:19:09,657] INFO [Controller id=6] New topics: [Set(mytopic)],
deleted topics: [Set()], new partition replica assignment [Map(mytopic-2 ->
Vector(6), mytopic-0 -> Vector(4), mytopic-1 -> Vector(5), mytopic-3 ->
Vector(7))] (kafka.controller.KafkaController)
[2018-01-09 11:19:09,658] INFO [Controller id=6] New topic creation callback
for mytopic-2,mytopic-0,mytopic-1,mytopic-3 (kafka.controller.KafkaController)
[2018-01-09 11:19:09,658] INFO [Controller id=6] New partition creation
callback for mytopic-2,mytopic-0,mytopic-1,mytopic-3
(kafka.controller.KafkaController)
[2018-01-09 11:19:09,658] INFO [PartitionStateMachine controllerId=6] Invoking
state change to NewPartition for partitions
mytopic-2,mytopic-0,mytopic-1,mytopic-3 (kafka.controller.PartitionStateMachine)
[2018-01-09 11:19:09,659] INFO [PartitionStateMachine controllerId=6] Invoking
state change to OnlinePartition for partitions
mytopic-2,mytopic-0,mytopic-1,mytopic-3 (kafka.controller.PartitionStateMachine)
[2018-01-09 11:19:09,664] WARN [OfflinePartitionLeaderSelector]: No broker in
ISR is alive for mytopic-0. Elect leader 4 from live brokers 4. There's
potential data loss. (kafka.controller.OfflinePartitionLeaderSelector)
[2018-01-09 11:19:09,664] INFO [OfflinePartitionLeaderSelector]: Selected new
leader and ISR {"leader":4,"leader_epoch":2,"isr":[4]} for offline partition
mytopic-0 (kafka.controller.OfflinePartitionLeaderSelector)
[2018-01-09 11:19:09,769] ERROR [Controller id=6] Forcing the controller to
resign (kafka.controller.KafkaController)
{code}
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)