Rob Young created KAFKA-19583: --------------------------------- Summary: Native docker image fails to start when using SASL OAUTHBEARER mechanism Key: KAFKA-19583 URL: https://issues.apache.org/jira/browse/KAFKA-19583 Project: Kafka Issue Type: Bug Components: docker Affects Versions: 4.0.0, 4.1.0 Environment: podman -v podman version 5.5.2
uname -r 6.15.8-200.fc42.x86_64 Reporter: Rob Young Running the native image with `KAFKA_SASL_ENABLED_MECHANISMS=OAUTHBEARER` results in an exception that prevents it starting: {code:java} [2025-08-06 22:55:09,656] ERROR Exiting Kafka due to fatal exception during startup. (kafka.Kafka$) org.apache.kafka.common.KafkaException: org.apache.kafka.common.KafkaException: Could not find a public no-argument constructor for org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredLoginCallbackHandler at org.apache.kafka.common.network.SaslChannelBuilder.configure(SaslChannelBuilder.java:184) ~[?:?] at org.apache.kafka.common.network.ChannelBuilders.create(ChannelBuilders.java:188) ~[?:?] at org.apache.kafka.common.network.ChannelBuilders.serverChannelBuilder(ChannelBuilders.java:105) ~[?:?] at kafka.network.Processor.<init>(SocketServer.scala:883) ~[?:?] at kafka.network.Acceptor.newProcessor(SocketServer.scala:791) ~[kafka.Kafka:?] at kafka.network.Acceptor.$anonfun$addProcessors$1(SocketServer.scala:757) ~[kafka.Kafka:?] at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:190) ~[kafka.Kafka:?] at kafka.network.Acceptor.addProcessors(SocketServer.scala:756) ~[kafka.Kafka:?] at kafka.network.DataPlaneAcceptor.configure(SocketServer.scala:472) ~[?:?] at kafka.network.SocketServer.createDataPlaneAcceptorAndProcessors(SocketServer.scala:222) ~[?:?] at kafka.network.SocketServer.$anonfun$new$16(SocketServer.scala:149) ~[?:?] at kafka.network.SocketServer.$anonfun$new$16$adapted(SocketServer.scala:149) ~[?:?] at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:619) ~[kafka.Kafka:?] at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:617) ~[kafka.Kafka:?] at scala.collection.AbstractIterable.foreach(Iterable.scala:935) ~[kafka.Kafka:?] at kafka.network.SocketServer.<init>(SocketServer.scala:149) ~[?:?] at kafka.server.BrokerServer.startup(BrokerServer.scala:274) ~[?:?] at kafka.server.KafkaRaftServer.$anonfun$startup$2(KafkaRaftServer.scala:96) ~[?:?] at kafka.server.KafkaRaftServer.$anonfun$startup$2$adapted(KafkaRaftServer.scala:96) ~[?:?] at scala.Option.foreach(Option.scala:437) ~[kafka.Kafka:?] at kafka.server.KafkaRaftServer.startup(KafkaRaftServer.scala:96) ~[?:?] at kafka.Kafka$.main(Kafka.scala:97) [kafka.Kafka:?] at kafka.docker.KafkaDockerWrapper$.main(KafkaDockerWrapper.scala:68) [kafka.Kafka:?] at kafka.docker.KafkaDockerWrapper.main(KafkaDockerWrapper.scala) [kafka.Kafka:?] at java.base/java.lang.invoke.LambdaForm$DMH/sa346b79c.invokeStaticInit(LambdaForm$DMH) [kafka.Kafka:?] Caused by: org.apache.kafka.common.KafkaException: Could not find a public no-argument constructor for org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredLoginCallbackHandler at org.apache.kafka.common.utils.Utils.newInstance(Utils.java:403) ~[?:?] at org.apache.kafka.common.security.authenticator.LoginManager.<init>(LoginManager.java:65) ~[?:?] at org.apache.kafka.common.security.authenticator.LoginManager.acquireLoginManager(LoginManager.java:123) ~[?:?] at org.apache.kafka.common.network.SaslChannelBuilder.configure(SaslChannelBuilder.java:170) ~[?:?] ... 24 more Caused by: java.lang.NoSuchMethodException: org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredLoginCallbackHandler.<init>() at java.base/java.lang.Class.checkMethod(DynamicHub.java:1075) ~[kafka.Kafka:?] at java.base/java.lang.Class.getConstructor0(DynamicHub.java:1238) ~[kafka.Kafka:?] at java.base/java.lang.Class.getDeclaredConstructor(DynamicHub.java:2930) ~[kafka.Kafka:?] at org.apache.kafka.common.utils.Utils.newInstance(Utils.java:401) ~[?:?] at org.apache.kafka.common.security.authenticator.LoginManager.<init>(LoginManager.java:65) ~[?:?] at org.apache.kafka.common.security.authenticator.LoginManager.acquireLoginManager(LoginManager.java:123) ~[?:?] at org.apache.kafka.common.network.SaslChannelBuilder.configure(SaslChannelBuilder.java:170) ~[?:?] ... 24 more{code} Here is a reproducer bash script: {code:java} temp_dir=$(mktemp -d) cd ${temp_dir} cat << EOF > kafka_server_jaas.conf KafkaServer { org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required unsecuredLoginStringClaim_sub=principal; }; EOF podman run -it --rm \ --name kafka-sasl-broker \ -p 9092:9092 \ -p 9093:9093 \ -v ./kafka_server_jaas.conf:/opt/kafka/config/kafka_server_jaas.conf:Z \ -e KAFKA_CLUSTER_ID=$KAFKA_CLUSTER_ID \ -e KAFKA_PROCESS_ROLES=broker,controller \ -e KAFKA_NODE_ID=1 \ -e KAFKA_LISTENER_SECURITY_PROTOCOL_MAP=SASL_PLAINTEXT:SASL_PLAINTEXT,INTER_BROKER:PLAINTEXT,CONTROLLER:PLAINTEXT \ -e KAFKA_LISTENERS=SASL_PLAINTEXT://0.0.0.0:9092,INTER_BROKER://0.0.0.0:9093,CONTROLLER://0.0.0.0:9094 \ -e KAFKA_ADVERTISED_LISTENERS=SASL_PLAINTEXT://localhost:9092,INTER_BROKER://localhost:9093 \ -e KAFKA_CONTROLLER_LISTENER_NAMES=CONTROLLER \ -e KAFKA_CONTROLLER_QUORUM_VOTERS=1@localhost:9094 \ -e KAFKA_INTER_BROKER_LISTENER_NAME=INTER_BROKER \ -e KAFKA_SASL_ENABLED_MECHANISMS=OAUTHBEARER \ -e KAFKA_OPTS="-Djava.security.auth.login.config=/opt/kafka/config/kafka_server_jaas.conf" \ apache/${1}{code} I guess some config is needed to avoid pruning out the reflectively instantiated class. `./run-oauthbearer.sh kafka-native:4.0.0` and `./run-oauthbearer.sh kafka-native:4.1.0-rc2` fail with exception. Running the same script with the main image works: `./run-oauthbearer.sh kafka:4.0.0` or `./run-oauthbearer.sh kafka:4.1.0-rc2`successfully start the broker. For context we want to use the image for integration testing and can workaround by running the non-native image if we need SASL oauthbearer. -- This message was sent by Atlassian Jira (v8.20.10#820010)