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

kraft.version/dirID is not updated after upgrading to MV 3.9

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • None
    • None
    • None
    • None

    Description

      When upgrading old controllers to v3.9.0, we didn't update the kraft.version to 1 and the directory ID, which causes it blocks the following controller addition/removal.

       

      Steps to reproduce:
      1. format 1 controller node in MV 3.8:

       bin/kafka-storage.sh format -t r0NrSlvGTqmZCtFtZ8tBIg -c config/kraft/controller1.properties --release-version 3.8 
      

      2. start up the controller
      3. update the metadata version to 3.9:

      bin/kafka-features.sh --bootstrap-controller localhost:9093 upgrade --metadata 3.9
      
      or
      
      bin/kafka-features.sh --bootstrap-controller localhost:9093 upgrade --feature kraft.version=1
      

      4. format one additional controller to be expanded

      bin/kafka-storage.sh format --cluster-id r0NrSlvGTqmZCtFtZ8tBIg --config config/kraft/controller2.properties --no-initial-controllers
      

      5. Try to add this node

      bin/kafka-metadata-quorum.sh --command-config config/kraft/controller2.properties --bootstrap-controller localhost:9093 add-controller
      

      It'll fail with:

      org.apache.kafka.common.errors.UnsupportedVersionException: Cluster doesn't support adding voter because the kraft.version feature is 0
      java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.UnsupportedVersionException: Cluster doesn't support adding voter because the kraft.version feature is 0
      	at java.base/java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:396)
      	at java.base/java.util.concurrent.CompletableFuture.get(CompletableFuture.java:2073)
      	at org.apache.kafka.common.internals.KafkaFutureImpl.get(KafkaFutureImpl.java:165)
      	at org.apache.kafka.tools.MetadataQuorumCommand.handleAddController(MetadataQuorumCommand.java:431)
      	at org.apache.kafka.tools.MetadataQuorumCommand.execute(MetadataQuorumCommand.java:147)
      	at org.apache.kafka.tools.MetadataQuorumCommand.mainNoExit(MetadataQuorumCommand.java:81)
      	at org.apache.kafka.tools.MetadataQuorumCommand.main(MetadataQuorumCommand.java:76)
      Caused by: org.apache.kafka.common.errors.UnsupportedVersionException: Cluster doesn't support adding voter because the kraft.version feature is 0
      

       

      Under this situation, I can't find any workaround to update kraft.version even if rebooting the node.

      Besides, we also didn't generate directory id for the upgraded nodes. Without the directory id, even if we have correct kraft.version, the controller removal still cannot succeed.

      ./bin/kafka-metadata-quorum.sh --bootstrap-server localhost:9092 describe --replication
      NodeId    DirectoryId               LogEndOffset    Lag    LastFetchTimestamp    LastCaughtUpTimestamp    Status      
      1         AAAAAAAAAAAAAAAAAAAAAA    588             0      1729752652123         1729752652123            Leader      
      4         AAAAAAAAAAAAAAAAAAAAAA    588             0      1729752651778         1729752651778            Follower    
      5         AAAAAAAAAAAAAAAAAAAAAA    588             0      1729752651782         1729752651782            Follower

       

      The directoryID `AAAAAAAAAAAAAAAAAAAAAA` in all 3 nodes means zero UUID (new Uuid(0L, 0L)), which is not correct.

       

      I think we should have a MetadataPublisher to handle the MV/feature upgrade for kraft.version and directory ID.

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              showuon Luke Chen
              Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: