From users-return-38859-apmail-kafka-users-archive=kafka.apache.org@kafka.apache.org Mon Nov 25 20:42:56 2019 Return-Path: X-Original-To: apmail-kafka-users-archive@www.apache.org Delivered-To: apmail-kafka-users-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [207.244.88.153]) by minotaur.apache.org (Postfix) with SMTP id 2C64619C75 for ; Mon, 25 Nov 2019 20:42:56 +0000 (UTC) Received: (qmail 95465 invoked by uid 500); 25 Nov 2019 20:42:46 -0000 Delivered-To: apmail-kafka-users-archive@kafka.apache.org Received: (qmail 95389 invoked by uid 500); 25 Nov 2019 20:42:46 -0000 Mailing-List: contact users-help@kafka.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: users@kafka.apache.org Delivered-To: mailing list users@kafka.apache.org Received: (qmail 95356 invoked by uid 99); 25 Nov 2019 20:42:45 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 25 Nov 2019 20:42:45 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id EE5E5C2DE8 for ; Mon, 25 Nov 2019 20:42:44 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.001 X-Spam-Level: X-Spam-Status: No, score=0.001 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, DKIM_VALID_AU=-0.1, DKIM_VALID_EF=-0.1, HTML_MESSAGE=0.2, RCVD_IN_DNSWL_NONE=-0.0001, SPF_HELO_NONE=0.001, SPF_PASS=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd4-us-west.apache.org (amavisd-new); dkim=pass (1024-bit key) header.d=easypost.com Received: from mx1-he-de.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id vFUcvsuO17P3 for ; Mon, 25 Nov 2019 20:42:39 +0000 (UTC) Received-SPF: Pass (mailfrom) identity=mailfrom; client-ip=2607:f8b0:4864:20::d2a; helo=mail-io1-xd2a.google.com; envelope-from=jbrown@easypost.com; receiver= Received: from mail-io1-xd2a.google.com (mail-io1-xd2a.google.com [IPv6:2607:f8b0:4864:20::d2a]) by mx1-he-de.apache.org (ASF Mail Server at mx1-he-de.apache.org) with ESMTPS id 312A87F51D for ; Mon, 25 Nov 2019 20:42:37 +0000 (UTC) Received: by mail-io1-xd2a.google.com with SMTP id j13so17898622ioe.0 for ; Mon, 25 Nov 2019 12:42:37 -0800 (PST) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=easypost.com; s=google; h=mime-version:from:date:message-id:subject:to; bh=YzYkSgiq3ERaDqDX2AChq7kArkVev1O+U9AeVDe90Zo=; b=EwSE64GH3E/aWb00dmUG7kYXnBrA2NdRbgx19HHg28RfV6/avXQD8og7G4rXd0BITj rx+mOWqibUKuhjeDARVqV84dwYV7rVu3pfw0C+C+ue0ba7syS8y5Ee5G44UVk98vLsLF PlEJradP2o0+yGU0860tPMYtGAcsClxdfcePM= X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:mime-version:from:date:message-id:subject:to; bh=YzYkSgiq3ERaDqDX2AChq7kArkVev1O+U9AeVDe90Zo=; b=idDstAVxaSR98rLXDRCZZorqB8NKFv4NFHC/iZqjDxkZaJk2JXoY4tOoWKnN6fxIcV E0yajWWWXVozaIBGiw2bHCKmGOF4b13gBzmyHVhv4f5zCSytX7+N4acVfEoKBmBNMftl 9N8TZtMnm534UpzFpkUAyRqlZoX3d16L4w8DcVTfzVvowytSWk5OwAZ/yPy240Kwtqm2 45mXIu49kdP2jWXUqAIZuf0WnO9OLL4dYzvdpIppHHdjCTT62HaBiXOws6luxj+gXZFJ c+HGo1T+s+H4gW5V/PVLoyfn+PHiv7vTAIt/FPCYrCryD+C2bkzAk05XsVUBnX7PFwsJ yMaw== X-Gm-Message-State: APjAAAXxF2SdSLI1zlx9mfFgt0Qsknhqh/bM9vinRu67Ff0w+TyQyRSt yJQFDm5fAccvXpn/CFnuQQqw088kyfGtBs7vQi+R6pWsiJdWDA== X-Google-Smtp-Source: APXvYqyLF+NbdKMiRGxTmbfBDrsC8L2Cy8nEti1zCSN8qhtOfzVwN4JrwVgg9nxRzA1ZODQcSTf00oM28XSEcbJeDz4= X-Received: by 2002:a5d:9592:: with SMTP id a18mr27159318ioo.186.1574714555474; Mon, 25 Nov 2019 12:42:35 -0800 (PST) MIME-Version: 1.0 From: James Brown Date: Mon, 25 Nov 2019 12:42:19 -0800 Message-ID: Subject: New topic gets stuck in bad state after Zookeeper error To: users@kafka.apache.org Content-Type: multipart/alternative; boundary="000000000000b3bf38059831cd35" --000000000000b3bf38059831cd35 Content-Type: text/plain; charset="UTF-8" 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 --000000000000b3bf38059831cd35--