Kafka
  1. Kafka
  2. KAFKA-150

Confusing NodeExistsException failing kafka broker startup

    Details

    • Type: Improvement Improvement
    • Status: Resolved
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.7
    • Fix Version/s: 0.7
    • Component/s: core
    • Labels:
      None

      Description

      Sometimes, broker startup fails with the following exception

      [2011-10-03 15:33:22,193] INFO Awaiting connections on port 9092
      (kafka.network.Acceptor)
      [2011-10-03 15:33:22,193] INFO Registering broker /brokers/ids/0
      (kafka.server.KafkaZooKeeper)
      [2011-10-03 15:33:22,229] INFO conflict in /brokers/ids/0 data:
      10.98.20.109-1317681202194:10.98.20.109:9092 stored data:
      10.98.20.109-1317268078266:10.98.20.109:9092 (kafka.utils.ZkUtils$)
      [2011-10-03 15:33:22,230] FATAL
      org.I0Itec.zkclient.exception.ZkNodeExistsException:
      org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode =
      NodeExists for /brokers/ids/0 (kafka.server.KafkaServer)
      [2011-10-03 15:33:22,231] FATAL
      org.I0Itec.zkclient.exception.ZkNodeExistsException:
      org.apache.zookeeper.KeeperException$NodeExistsException: KeeperErrorCode =
      NodeExists for /brokers/ids/0
      at org.I0Itec.zkclient.exception.ZkException.create(ZkException.java:55)
      at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:685)
      at org.I0Itec.zkclient.ZkClient.create(ZkClient.java:304)
      at org.I0Itec.zkclient.ZkClient.createEphemeral(ZkClient.java:328)
      at kafka.utils.ZkUtils$.createEphemeralPath(ZkUtils.scala:55)
      at
      kafka.utils.ZkUtils$.createEphemeralPathExpectConflict(ZkUtils.scala:71)
      at
      kafka.server.KafkaZooKeeper.registerBrokerInZk(KafkaZooKeeper.scala:54)
      at kafka.log.LogManager.startup(LogManager.scala:122)
      at kafka.server.KafkaServer.startup(KafkaServer.scala:77)
      at
      kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:40)
      at kafka.Kafka$.main(Kafka.scala:56)
      at kafka.Kafka.main(Kafka.scala)
      Caused by: org.apache.zookeeper.KeeperException$NodeExistsException:
      KeeperErrorCode = NodeExists for /brokers/ids/0
      at org.apache.zookeeper.KeeperException.create(KeeperException.java:110)
      at org.apache.zookeeper.KeeperException.create(KeeperException.java:42)
      at org.apache.zookeeper.ZooKeeper.create(ZooKeeper.java:637)
      at org.I0Itec.zkclient.ZkConnection.create(ZkConnection.java:87)
      at org.I0Itec.zkclient.ZkClient$1.call(ZkClient.java:308)
      at org.I0Itec.zkclient.ZkClient$1.call(ZkClient.java:304)
      at org.I0Itec.zkclient.ZkClient.retryUntilConnected(ZkClient.java:675)
      ... 10 more
      (kafka.server.KafkaServer)
      [2011-10-03 15:33:22,231] INFO Shutting down... (kafka.server.KafkaServer)
      [2011-10-03 15:33:22,232] INFO shutdown scheduler kafka-logcleaner-
      (kafka.utils.KafkaScheduler)
      [2011-10-03 15:33:22,239] INFO shutdown scheduler kafka-logflusher-
      (kafka.utils.KafkaScheduler)
      [2011-10-03 15:33:22,481] INFO zkActor stopped (kafka.log.LogManager)
      [2011-10-03 15:33:22,482] INFO Closing zookeeper client...
      (kafka.server.KafkaZooKeeper)
      [2011-10-03 15:33:22,482] INFO Terminate ZkClient event thread.
      (org.I0Itec.zkclient.ZkEventThread)

      There could be 3 things that might have happened
      (1) you restarted kafka within the zk timeout, in which case as far as zk is concerned your old broker still exists...this is weird but actually correct behavior,
      (2) you have two brokers with the same id,
      (3) zk has a bug and is not deleting ephemeral nodes.

      Instead of just throwing the ZK NodeExistsException, we should include the above information in a well-named Kafka exception, for clarity.

        Activity

        Hide
        Neha Narkhede added a comment -

        Good point. Changed the fixed version to 0.7

        Show
        Neha Narkhede added a comment - Good point. Changed the fixed version to 0.7
        Hide
        Chris Burroughs added a comment -

        trunk is still 0.7, right?

        Show
        Chris Burroughs added a comment - trunk is still 0.7, right?
        Hide
        Jun Rao added a comment -

        +1 on the patch.

        I thought the try/catch thing in scala is just a special case of match/case.

        Show
        Jun Rao added a comment - +1 on the patch. I thought the try/catch thing in scala is just a special case of match/case.
        Hide
        Jay Kreps added a comment -

        Hmm, good thinking. But actually I don't think that is how try/catch works in scala. It just catches the exception you specify. So what I have there is equivalent to
        try

        { ... }

        catch (ZkNodeExistsException e)

        {...}

        I tested this out just to be sure, and it does seem to work that way (which is good because otherwise every single try/catch would have to remember to rethrow).

        scala> jkreps-mn:kafka-trunk jkreps$ scala
        Welcome to Scala version 2.8.0.final (Java HotSpot(TM) 64-Bit Server VM, Java 1.6.0_22).
        Type in expressions to have them evaluated.
        Type :help for more information.

        scala> try

        { | throw new Exception("") | }

        catch

        { | case e: RuntimeException => println("gotcha") | }

        java.lang.Exception:
        at .liftedTree1$1(<console>:7)
        at .<init>(<console>:6)
        at .<clinit>(<console>)
        at RequestResult$.<init>(<console>:9)
        at RequestResult$.<clinit>(<console>)
        at RequestResult$scala_repl_result(<console>)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
        at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
        at java.lang.reflect.Method.invoke(Method.java:597)
        at scala.tools.nsc.Interpreter$Request$$anonfun$loadAndRun$1$$anonfun$apply$18.apply(Interpreter.scala:981)
        at scala.tools.nsc.Interpreter$Request$$anonfun$loadAndRun$1$$anonfun$apply$18.apply(Interpreter.scala:981)
        at scala.util.control.Exception$Catch.apply(Exception.scala:...

        Show
        Jay Kreps added a comment - Hmm, good thinking. But actually I don't think that is how try/catch works in scala. It just catches the exception you specify. So what I have there is equivalent to try { ... } catch (ZkNodeExistsException e) {...} I tested this out just to be sure, and it does seem to work that way (which is good because otherwise every single try/catch would have to remember to rethrow). scala> jkreps-mn:kafka-trunk jkreps$ scala Welcome to Scala version 2.8.0.final (Java HotSpot(TM) 64-Bit Server VM, Java 1.6.0_22). Type in expressions to have them evaluated. Type :help for more information. scala> try { | throw new Exception("") | } catch { | case e: RuntimeException => println("gotcha") | } java.lang.Exception: at .liftedTree1$1(<console>:7) at .<init>(<console>:6) at .<clinit>(<console>) at RequestResult$.<init>(<console>:9) at RequestResult$.<clinit>(<console>) at RequestResult$scala_repl_result(<console>) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at scala.tools.nsc.Interpreter$Request$$anonfun$loadAndRun$1$$anonfun$apply$18.apply(Interpreter.scala:981) at scala.tools.nsc.Interpreter$Request$$anonfun$loadAndRun$1$$anonfun$apply$18.apply(Interpreter.scala:981) at scala.util.control.Exception$Catch.apply(Exception.scala:...
        Hide
        Jun Rao added a comment -

        In KafkaZooKeeper.registerBrokerInZk, in the case statement, we need to rethrow exceptions other than ZkNodeExistsException.

        Show
        Jun Rao added a comment - In KafkaZooKeeper.registerBrokerInZk, in the case statement, we need to rethrow exceptions other than ZkNodeExistsException.
        Hide
        Jay Kreps added a comment -

        Now produces the following error:

        [2011-10-23 21:16:53,635] FATAL Fatal error during startup. (kafka.server.KafkaServer)
        java.lang.RuntimeException: A broker is already registered on the path /brokers/ids/0. This probably indicates that you either have configured a brokerid that is already in use, or else you have shutdown this broker and restarted it faster than the zookeeper timeout so it appears to be re-registering.
        at kafka.server.KafkaZooKeeper.registerBrokerInZk(KafkaZooKeeper.scala:60)
        at kafka.log.LogManager.startup(LogManager.scala:123)
        at kafka.server.KafkaServer.startup(KafkaServer.scala:79)
        at kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:46)
        at kafka.Kafka$.main(Kafka.scala:60)
        at kafka.Kafka.main(Kafka.scala)

        Show
        Jay Kreps added a comment - Now produces the following error: [2011-10-23 21:16:53,635] FATAL Fatal error during startup. (kafka.server.KafkaServer) java.lang.RuntimeException: A broker is already registered on the path /brokers/ids/0. This probably indicates that you either have configured a brokerid that is already in use, or else you have shutdown this broker and restarted it faster than the zookeeper timeout so it appears to be re-registering. at kafka.server.KafkaZooKeeper.registerBrokerInZk(KafkaZooKeeper.scala:60) at kafka.log.LogManager.startup(LogManager.scala:123) at kafka.server.KafkaServer.startup(KafkaServer.scala:79) at kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:46) at kafka.Kafka$.main(Kafka.scala:60) at kafka.Kafka.main(Kafka.scala)

          People

          • Assignee:
            Jay Kreps
            Reporter:
            Neha Narkhede
          • Votes:
            0 Vote for this issue
            Watchers:
            0 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development