Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-2818

Clean up Controller Object on forced Resignation

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 0.9.0.0
    • 0.11.0.0
    • controller
    • None

    Description

      Currently if the controller does a forced resignation (if an exception is caught during updateLeaderEpochAndSendRequest, SendUpdateMetadataRequest or shutdownBroker), the Zookeeper resignation callback function OnControllerResignation doesn't get a chance to execute which leaves some artifacts laying around. In particular the Sensors dont get cleaned up and if the Kafka broker ever gets re-elected as Controller it will fail due to some metrics already existing. An Error and stack trace of such an event is below.

      A forced resignation situation can be induced with a mis-config in broker.properties fairly easily, by settig only SASL_PLAINTTEXT listeners and setting inter.broker.protocol.version=0.8.2.X

      listeners=SASL_PLAINTEXT://<HOST FQDN>:9092
      inter.broker.protocol.version=0.8.2.X
      security.inter.broker.protocol=SASL_PLAINTEXT
      
      [2015-11-09 16:33:47,510] ERROR Error while electing or becoming leader on broker 182050300 (kafka.server.ZookeeperLeaderElector)
      java.lang.IllegalArgumentException: A metric named 'MetricName [name=connection-close-rate, group=controller-channel-metrics, description=Connections closed per second in the window., tags={broker-id=182050300}]' already exists, can't register another one.
              at org.apache.kafka.common.metrics.Metrics.registerMetric(Metrics.java:285)
              at org.apache.kafka.common.metrics.Sensor.add(Sensor.java:177)
              at org.apache.kafka.common.metrics.Sensor.add(Sensor.java:162)
              at org.apache.kafka.common.network.Selector$SelectorMetrics.<init>(Selector.java:578)
              at org.apache.kafka.common.network.Selector.<init>(Selector.java:112)
              at kafka.controller.ControllerChannelManager.kafka$controller$ControllerChannelManager$$addNewBroker(ControllerChannelManager.scala:91)
              at kafka.controller.ControllerChannelManager$$anonfun$1.apply(ControllerChannelManager.scala:43)
              at kafka.controller.ControllerChannelManager$$anonfun$1.apply(ControllerChannelManager.scala:43)
              at scala.collection.immutable.Set$Set1.foreach(Set.scala:74)
              at kafka.controller.ControllerChannelManager.<init>(ControllerChannelManager.scala:43)
              at kafka.controller.KafkaController.startChannelManager(KafkaController.scala:819)
              at kafka.controller.KafkaController.initializeControllerContext(KafkaController.scala:747)
              at kafka.controller.KafkaController.onControllerFailover(KafkaController.scala:330)
              at kafka.controller.KafkaController$$anonfun$1.apply$mcV$sp(KafkaController.scala:163)
              at kafka.server.ZookeeperLeaderElector.elect(ZookeeperLeaderElector.scala:84)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply$mcZ$sp(ZookeeperLeaderElector.scala:146)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener$$anonfun$handleDataDeleted$1.apply(ZookeeperLeaderElector.scala:141)
              at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:262)
              at kafka.server.ZookeeperLeaderElector$LeaderChangeListener.handleDataDeleted(ZookeeperLeaderElector.scala:141)
              at org.I0Itec.zkclient.ZkClient$9.run(ZkClient.java:823)
              at org.I0Itec.zkclient.ZkEventThread.run(ZkEventThread.java:71)
      

      Attachments

        1. KAFKA-2818.patch
          2 kB
          Matthew Bruce

        Issue Links

          Activity

            People

              fpj Flavio Paiva Junqueira
              mbruce@blackberry.com Matthew Bruce
              Votes:
              1 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: