From jira-return-8685-archive-asf-public=cust-asf.ponee.io@kafka.apache.org Wed Jan 10 18:09:06 2018 Return-Path: X-Original-To: archive-asf-public@eu.ponee.io Delivered-To: archive-asf-public@eu.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by mx-eu-01.ponee.io (Postfix) with ESMTP id ED41818072F for ; Wed, 10 Jan 2018 18:09:05 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id DD822160C2E; Wed, 10 Jan 2018 17:09:05 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id E7D7C160C23 for ; Wed, 10 Jan 2018 18:09:04 +0100 (CET) Received: (qmail 19158 invoked by uid 500); 10 Jan 2018 17:09:04 -0000 Mailing-List: contact jira-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@kafka.apache.org Delivered-To: mailing list jira@kafka.apache.org Received: (qmail 19147 invoked by uid 99); 10 Jan 2018 17:09:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 10 Jan 2018 17:09:04 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id ACA62C3624 for ; Wed, 10 Jan 2018 17:09:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.711 X-Spam-Level: X-Spam-Status: No, score=-100.711 tagged_above=-999 required=6.31 tests=[RCVD_IN_DNSWL_LOW=-0.7, SPF_PASS=-0.001, T_RP_MATCHES_RCVD=-0.01, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id DwVuMEbem6h3 for ; Wed, 10 Jan 2018 17:09:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id 01D5A5FB69 for ; Wed, 10 Jan 2018 17:09:02 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id C078FE0373 for ; Wed, 10 Jan 2018 17:09:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 81AD2274D6 for ; Wed, 10 Jan 2018 17:09:00 +0000 (UTC) Date: Wed, 10 Jan 2018 17:09:00 +0000 (UTC) From: "Ismael Juma (JIRA)" To: jira@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (KAFKA-6438) NSEE while concurrently creating and deleting a topic MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/KAFKA-6438?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ismael Juma updated KAFKA-6438: ------------------------------- Fix Version/s: 1.1.0 > 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 > Labels: reliability > Fix For: 1.1.0 > > > 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, other2)], 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)