Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id A7E6B200CAC for ; Mon, 19 Jun 2017 13:16:08 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id A6736160BE4; Mon, 19 Jun 2017 11:16:08 +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 C3ABC160BE1 for ; Mon, 19 Jun 2017 13:16:07 +0200 (CEST) Received: (qmail 89433 invoked by uid 500); 19 Jun 2017 11:16: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 89422 invoked by uid 99); 19 Jun 2017 11:16:04 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 19 Jun 2017 11:16:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 992F21AF911 for ; Mon, 19 Jun 2017 11:16:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.202 X-Spam-Level: X-Spam-Status: No, score=-99.202 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id vI3L5ejIS06p for ; Mon, 19 Jun 2017 11:16:02 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id B660B5FC99 for ; Mon, 19 Jun 2017 11:16:01 +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 172F1E0DA7 for ; Mon, 19 Jun 2017 11:16:01 +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 3BACC24005 for ; Mon, 19 Jun 2017 11:16:00 +0000 (UTC) Date: Mon, 19 Jun 2017 11:16:00 +0000 (UTC) From: "Pengwei (JIRA)" To: jira@kafka.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (KAFKA-4900) Brokers stuck in controller re-election loop after failing to register metrics MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 19 Jun 2017 11:16:08 -0000 [ https://issues.apache.org/jira/browse/KAFKA-4900?page=3Dcom.atlassian= .jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D1605= 3813#comment-16053813 ]=20 Pengwei commented on KAFKA-4900: -------------------------------- [~ijuma] This PR is for the https://issues.apache.org/jira/browse/KAFKA-376= 1, and Kafka-3761 is fixed in 0.10.1.0. But I found that in the 0.10.2.0=EF=BC=8C the ZookeeperLeaderElector.resign= still setting the leaderId to -1 > Brokers stuck in controller re-election loop after failing to register me= trics > -------------------------------------------------------------------------= ----- > > Key: KAFKA-4900 > URL: https://issues.apache.org/jira/browse/KAFKA-4900 > Project: Kafka > Issue Type: Bug > Components: controller, core > Affects Versions: 0.10.1.1 > Reporter: Nick Travers > > We hit this today in one of out three node staging clusters. The exceptio= n continues to occur on all three nodes. > {code} > 2017-03-15 02:17:30,677 ERROR [ZkClient-EventThread-35-samsa-zkserver.sta= ge.sjc1.square:26101/samsa] server.ZookeeperLeaderElector - Error while ele= cting or becoming leader on broker 9 > java.lang.IllegalArgumentException: A metric named 'MetricName [name=3Dco= nnection-close-rate, group=3Dcontroller-channel-metrics,description=3DConne= ctions closed per second in the window., tags=3D{broker-id=3D10}]' already = exists, can't register another one. > at org.apache.kafka.common.metrics.Metrics.registerMetric(Metrics= .java:380) > at org.apache.kafka.common.metrics.Sensor.add(Sensor.java:179) > at org.apache.kafka.common.metrics.Sensor.add(Sensor.java:164) > at org.apache.kafka.common.network.Selector$SelectorMetrics.(Selector.java:617) > at org.apache.kafka.common.network.Selector.(Selector.java:= 138) > at kafka.controller.ControllerChannelManager.kafka$controller$Con= trollerChannelManager$$addNewBroker(ControllerChannelManager.scala:101) > at kafka.controller.ControllerChannelManager$$anonfun$1.apply(Con= trollerChannelManager.scala:45) > at kafka.controller.ControllerChannelManager$$anonfun$1.apply(Con= trollerChannelManager.scala:45) > at scala.collection.immutable.Set$Set3.foreach(Set.scala:163) > at kafka.controller.ControllerChannelManager.(ControllerCha= nnelManager.scala:45) > at kafka.controller.KafkaController.startChannelManager(KafkaCont= roller.scala:814) > at kafka.controller.KafkaController.initializeControllerContext(K= afkaController.scala:742) > at kafka.controller.KafkaController.onControllerFailover(KafkaCon= troller.scala:334) > at kafka.controller.KafkaController$$anonfun$1.apply$mcV$sp(Kafka= Controller.scala:167) > at kafka.server.ZookeeperLeaderElector.elect(ZookeeperLeaderElect= or.scala:84) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anon= fun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:146) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anon= fun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anon= fun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234) > at kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handl= eDataDeleted(ZookeeperLeaderElector.scala:141) > at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:824) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)= =20 > {code} > We observe a tight loop of controller (re-)election, i.e. one node hits t= his exception, and leadership transitions to the next, which then hits the = exception, ad infinitum. > Producers and consumers appear to be connecting ok, and are able to produ= ce and consume messages. > Relevant data points: > - prior to this cluster restart, a partition reassignment was attempted = for a number of topics, which appeared to get stuck in the "in progress" st= ate (on the order of days) > - these topics were subsequently deleted > - a rolling restart of the cluster was performed was to turn on broker-t= o-broker SSL communication > - the SSL change has subsequently been _rolled back_ after we observed t= hese exceptions > - the entire cluster was shut down, and nodes brought back one at a time= in an attempt to clear the exception. We were able to restart the cluster,= but we continue to see the exceptions > We also observed, during the same time as the exception above, the follow= ing exception on all hosts: > {code} > 2017-03-15 01:44:04,572 ERROR [ZkClient-EventThread-36-samsa-zkserver.sta= ge.sjc1.square:26101/samsa] controller.ReplicaStateMachine$BrokerChangeList= ener - [BrokerChangeListener on Controller 10]: Error while handling broker= changes > java.lang.ClassCastException: java.lang.String cannot be cast to java.lan= g.Integer > at scala.runtime.BoxesRunTime.unboxToInt(BoxesRunTime.java:101) > at kafka.controller.KafkaController$$anonfun$8$$anonfun$apply$2.a= pply(KafkaController.scala:436) > at scala.collection.LinearSeqOptimized$class.exists(LinearSeqOpti= mized.scala:93) > at scala.collection.immutable.List.exists(List.scala:84) > at kafka.controller.KafkaController$$anonfun$8.apply(KafkaControl= ler.scala:436) > at kafka.controller.KafkaController$$anonfun$8.apply(KafkaControl= ler.scala:435) > at scala.collection.TraversableLike$$anonfun$filterImpl$1.apply(T= raversableLike.scala:248) > at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(Hash= Map.scala:99) > at scala.collection.mutable.HashMap$$anonfun$foreach$1.apply(Hash= Map.scala:99) > at scala.collection.mutable.HashTable$class.foreachEntry(HashTabl= e.scala:230) > at scala.collection.mutable.HashMap.foreachEntry(HashMap.scala:40= ) > at scala.collection.mutable.HashMap.foreach(HashMap.scala:99) > at scala.collection.TraversableLike$class.filterImpl(TraversableL= ike.scala:247) > at scala.collection.TraversableLike$class.filter(TraversableLike.= scala:259) > at scala.collection.AbstractTraversable.filter(Traversable.scala:= 104) > at kafka.controller.KafkaController.onBrokerStartup(KafkaControll= er.scala:435) > at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$ano= nfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply$mcV$sp(ReplicaStateM= achine.scala:374) > at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$ano= nfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.= scala:358) > at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$ano= nfun$handleChildChange$1$$anonfun$apply$mcV$sp$1.apply(ReplicaStateMachine.= scala:358) > at kafka.metrics.KafkaTimer.time(KafkaTimer.scala:33) > at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$ano= nfun$handleChildChange$1.apply$mcV$sp(ReplicaStateMachine.scala:357) > at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$ano= nfun$handleChildChange$1.apply(ReplicaStateMachine.scala:356) > at kafka.controller.ReplicaStateMachine$BrokerChangeListener$$ano= nfun$handleChildChange$1.apply(ReplicaStateMachine.scala:356) > at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:234) > at kafka.controller.ReplicaStateMachine$BrokerChangeListener.hand= leChildChange(ReplicaStateMachine.scala:355) > at org.I0Itec.zkclient.ZkClient$10.run(ZkClient.java:843) > at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71) > {code} -- This message was sent by Atlassian JIRA (v6.4.14#64029)