Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-14304 ZooKeeper to KRaft Migration
  3. KAFKA-14833

ZK migration cannot start if a ZK controller was not previously elected

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Minor
    • Resolution: Fixed
    • 3.4.0
    • None
    • kraft
    • None

    Description

      During the BecomeZkController event in KRaftMigrationDriver, an error occurs if there was not data in the /controller_epoch znode.

      [2023-03-22 14:33:06,082] ERROR Had an exception in BecomeZkControllerEvent (org.apache.kafka.metadata.migration.KRaftMigrationDriver)
      java.lang.IllegalStateException: Cannot register KRaft controller 3001 as the active controller since there is no ZK controller epoch present.
              at kafka.zk.KafkaZkClient.tryRegisterKRaftControllerAsActiveController(KafkaZkClient.scala:201)
              at kafka.zk.ZkMigrationClient.claimControllerLeadership(ZkMigrationClient.scala:60)
              at org.apache.kafka.metadata.migration.KRaftMigrationDriver.apply(KRaftMigrationDriver.java:143)
              at org.apache.kafka.metadata.migration.KRaftMigrationDriver.access$800(KRaftMigrationDriver.java:57)
              at org.apache.kafka.metadata.migration.KRaftMigrationDriver$BecomeZkControllerEvent.run(KRaftMigrationDriver.java:357)
              at org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:121)
              at org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:201)
              at org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:175)
              at java.lang.Thread.run(Thread.java:750)
      

      This mainly affects tests when we start a KRaft quorum first, and then the (new) ZK cluster. The migration driver will kick off when the ZK brokers register but it will race with KafkaController on the /controller_epoch creation.

      Attachments

        Activity

          People

            zhaohaidao HaiyuanZhao
            mumrah David Arthur
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: