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

Listener not found error in SendRPCsToBrokersEvent

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • None
    • 3.9.0
    • controller, migration
    • None

    Description

      During the ZK to KRaft migration, the controller will send RPCs to the ZK brokers using the configured "control.plane.listener.name" or more commonly, the "inter.broker.listener.name". If a ZK broker did not register with this listener, we get a error at the time of sending the first RPC to a broker.

      [2024-04-03 09:28:59,043] ERROR Encountered nonFatalFaultHandler fault: Unhandled error in SendRPCsToBrokersEvent (org.apache.kafka.server.fault.MockFaultHandler:44)
      kafka.common.BrokerEndPointNotAvailableException: End point with listener name EXTERNAL not found for broker 0
      	at kafka.cluster.Broker.$anonfun$node$1(Broker.scala:94)
      	at scala.Option.getOrElse(Option.scala:201)
      	at kafka.cluster.Broker.node(Broker.scala:93)
      	at kafka.controller.ControllerChannelManager.addNewBroker(ControllerChannelManager.scala:122)
      	at kafka.controller.ControllerChannelManager.addBroker(ControllerChannelManager.scala:105)
      	at kafka.migration.MigrationPropagator.$anonfun$publishMetadata$2(MigrationPropagator.scala:98)
      	at kafka.migration.MigrationPropagator.$anonfun$publishMetadata$2$adapted(MigrationPropagator.scala:98)
      	at scala.collection.immutable.Set$Set3.foreach(Set.scala:261)
      	at kafka.migration.MigrationPropagator.publishMetadata(MigrationPropagator.scala:98)
      	at kafka.migration.MigrationPropagator.sendRPCsToBrokersFromMetadataImage(MigrationPropagator.scala:219)
      	at org.apache.kafka.metadata.migration.KRaftMigrationDriver$SendRPCsToBrokersEvent.run(KRaftMigrationDriver.java:777)
      	at org.apache.kafka.queue.KafkaEventQueue$EventContext.run(KafkaEventQueue.java:128)
      	at org.apache.kafka.queue.KafkaEventQueue$EventHandler.handleEvents(KafkaEventQueue.java:211)
      	at org.apache.kafka.queue.KafkaEventQueue$EventHandler.run(KafkaEventQueue.java:182)
      	at java.base/java.lang.Thread.run(Thread.java:833)
      

      This is pretty late to be detecting this mis-configuration. By this point, the KRaft controller is the active controller and has already migrated the metadata. Recovery is possible by restarting the brokers with the correct listener names, but we can catch this much sooner in the process.

      When a ZK broker registers with the KRaft controller, we should reject the registration if the expected listener name is not present. This will prevent the migration from starting.

      Attachments

        Issue Links

          Activity

            People

              cmccabe Colin McCabe
              davidarthur David Arthur
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: