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

Kafka upgrade fails from 1.1 to 2.4/2.5/trunk fails due to failure in ZooKeeper

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 2.4.0, 2.5.0, 2.6.0
    • None
    • zkclient
    • None

    Description

      When we tested upgrading Kafka from 1.1 to 2.4/2.5, the upgraded node failed to start due to a known zookeeper failure - ZOOKEEPER-3056.

      The error message is shown below:

       

      [2020-05-24 23:45:17,638] ERROR Unexpected exception, exiting abnormally (org.apache.zookeeper.server.ZooKeeperServerMain)
      java.io.IOException: No snapshot found, but there are log entries. Something is broken!
       at org.apache.zookeeper.server.persistence.FileTxnSnapLog.restore(FileTxnSnapLog.java:240)
       at org.apache.zookeeper.server.ZKDatabase.loadDataBase(ZKDatabase.java:240)
       at org.apache.zookeeper.server.ZooKeeperServer.loadData(ZooKeeperServer.java:290)
       at org.apache.zookeeper.server.ZooKeeperServer.startdata(ZooKeeperServer.java:450)
       at org.apache.zookeeper.server.NIOServerCnxnFactory.startup(NIOServerCnxnFactory.java:764)
       at org.apache.zookeeper.server.ServerCnxnFactory.startup(ServerCnxnFactory.java:98)
       at org.apache.zookeeper.server.ZooKeeperServerMain.runFromConfig(ZooKeeperServerMain.java:144)
       at org.apache.zookeeper.server.ZooKeeperServerMain.initializeAndRun(ZooKeeperServerMain.java:106)
       at org.apache.zookeeper.server.ZooKeeperServerMain.main(ZooKeeperServerMain.java:64)
       at org.apache.zookeeper.server.quorum.QuorumPeerMain.initializeAndRun(QuorumPeerMain.java:128)
       at org.apache.zookeeper.server.quorum.QuorumPeerMain.main(QuorumPeerMain.java:82)
      

       

      [2020-05-24 23:45:25,142] ERROR Fatal error during KafkaServer startup. Prepare to shutdown (kafka.server.KafkaServer)
      kafka.zookeeper.ZooKeeperClientTimeoutException: Timed out waiting for connection while in state: CONNECTING
       at kafka.zookeeper.ZooKeeperClient.$anonfun$waitUntilConnected$3(ZooKeeperClient.scala:259)
       at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)
       at kafka.utils.CoreUtils$.inLock(CoreUtils.scala:253)
       at kafka.zookeeper.ZooKeeperClient.waitUntilConnected(ZooKeeperClient.scala:255)
       at kafka.zookeeper.ZooKeeperClient.<init>(ZooKeeperClient.scala:113)
       at kafka.zk.KafkaZkClient$.apply(KafkaZkClient.scala:1858)
       at kafka.server.KafkaServer.createZkClient$1(KafkaServer.scala:375)
       at kafka.server.KafkaServer.initZkClient(KafkaServer.scala:399)
       at kafka.server.KafkaServer.startup(KafkaServer.scala:207)
       at kafka.server.KafkaServerStartable.startup(KafkaServerStartable.scala:44)
       at kafka.Kafka$.main(Kafka.scala:84)
       at kafka.Kafka.main(Kafka.scala)

      It can be reproduced through the following steps:

      1. Start a single-node kafka 1.1.
      2. Create a topic and use kafka-producer-perf-test.sh to produce several message.

      bin/kafka-topics.sh --create --bootstrap-server localhost:9092 --replication-factor 1 --partitions 1 --topic test 
      bin/kafka-producer-perf-test.sh --topic test --num-records 500 --record-size 300 --throughput 100 --producer-props bootstrap.servers=localhost:9092

      3. Upgrade the node to 2.4/2.5 with the same configuration. The new version node failed to start because of the zookeeper.

      Kafka 1.1 is using dependant-libs-2.11.12/zookeeper-3.4.10.jar, and Kafka 2.4/2.5/trunk(5302efb2d1b7a69bcd3173a13b2d08a2666979ed) are using zookeeper-3.5.8.jar

      The bug is fixed in zookeeper-3.6.0, should we upgrade the dependency of Kafka 2.4/2.5/trunk to use zookeeper-3.6.0.jar?

      Attachments

        Activity

          People

            Unassigned Unassigned
            Zhuqi1108 Zhuqi Jin
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: