Daniel Urban created KAFKA-14721: ------------------------------------ Summary: Kafka listener uses wrong login class Key: KAFKA-14721 URL: https://issues.apache.org/jira/browse/KAFKA-14721 Project: Kafka Issue Type: Bug Affects Versions: 3.1.2 Reporter: Daniel Urban
When trying to configure a single SASL_SSL listener with GSSAPI, Scram and OAuth support, we encounter an error at startup: {code:java} 2023-02-15 13:26:04,250 ERROR kafka.server.KafkaServer: [main]: [KafkaServer id=104] Fatal error during KafkaServer startup. Prepare to shutdown org.apache.kafka.common.KafkaException: java.lang.IllegalArgumentException: No serviceName defined in either JAAS or Kafka config at org.apache.kafka.common.network.SaslChannelBuilder.configure(SaslChannelBuilder.java:184) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] at org.apache.kafka.common.network.ChannelBuilders.create(ChannelBuilders.java:192) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] at org.apache.kafka.common.network.ChannelBuilders.serverChannelBuilder(ChannelBuilders.java:107) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] at kafka.network.Processor.<init>(SocketServer.scala:861) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at kafka.network.SocketServer.newProcessor(SocketServer.scala:442) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at kafka.network.SocketServer.$anonfun$addDataPlaneProcessors$1(SocketServer.scala:299) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at scala.collection.immutable.Range.foreach$mVc$sp(Range.scala:190) ~[scala-library-2.13.10.jar:?] at kafka.network.SocketServer.addDataPlaneProcessors(SocketServer.scala:297) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at kafka.network.SocketServer.$anonfun$createDataPlaneAcceptorsAndProcessors$1(SocketServer.scala:262) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at kafka.network.SocketServer.$anonfun$createDataPlaneAcceptorsAndProcessors$1$adapted(SocketServer.scala:259) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at scala.collection.IterableOnceOps.foreach(IterableOnce.scala:575) ~[scala-library-2.13.10.jar:?] at scala.collection.IterableOnceOps.foreach$(IterableOnce.scala:573) ~[scala-library-2.13.10.jar:?] at scala.collection.AbstractIterable.foreach(Iterable.scala:933) ~[scala-library-2.13.10.jar:?] at kafka.network.SocketServer.createDataPlaneAcceptorsAndProcessors(SocketServer.scala:259) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at kafka.network.SocketServer.startup(SocketServer.scala:131) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at kafka.server.KafkaServer.startup(KafkaServer.scala:310) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at kafka.Kafka$.main(Kafka.scala:109) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at com.cloudera.kafka.wrap.Kafka$.$anonfun$main$1(Kafka.scala:107) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at com.cloudera.kafka.wrap.Kafka$.$anonfun$main$1$adapted(Kafka.scala:107) ~[kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at com.cloudera.kafka.wrap.Kafka$.runMain(Kafka.scala:118) [kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at com.cloudera.kafka.wrap.Kafka$.main(Kafka.scala:110) [kafka_2.13-3.1.2.7.1.9.0-15.jar:?] at com.cloudera.kafka.wrap.Kafka.main(Kafka.scala) [kafka_2.13-3.1.2.7.1.9.0-15.jar:?] Caused by: java.lang.IllegalArgumentException: No serviceName defined in either JAAS or Kafka config at org.apache.kafka.common.security.kerberos.KerberosLogin.getServiceName(KerberosLogin.java:309) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] at org.apache.kafka.common.security.kerberos.KerberosLogin.configure(KerberosLogin.java:92) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] at org.apache.kafka.common.security.authenticator.LoginManager.<init>(LoginManager.java:61) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] at org.apache.kafka.common.security.authenticator.LoginManager.acquireLoginManager(LoginManager.java:105) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] at org.apache.kafka.common.network.SaslChannelBuilder.configure(SaslChannelBuilder.java:170) ~[kafka-clients-3.1.2.7.1.9.0-15.jar:?] ... 21 more{code} Using the following configs in a Kafka broker: jaas configuration file: {code:java} KafkaServer { com.sun.security.auth.module.Krb5LoginModule required doNotPrompt=true useKeyTab=true storeKey=true serviceName="kafka" keyTab="/var/KAFKA_BROKER/kafka.keytab" principal="kafka/hgiovr@SITE"; org.apache.kafka.common.security.scram.ScramLoginModule required; };{code} and the following properties: {code:java} listener.name.sasl_ssl.sasl.enabled.mechanisms=GSSAPI,SCRAM-SHA-256,SCRAM-SHA-512,OAUTHBEARER listener.name.sasl_ssl.oauthbearer.sasl.jaas.config=org.apache.kafka.common.security.oauthbearer.OAuthBearerLoginModule required; listener.name.sasl_ssl.oauthbearer.sasl.server.callback.handler.class=org.apache.kafka.common.security.oauthbearer.secured.OAuthBearerValidatorCallbackHand ler sasl.oauthbearer.jwks.endpoint.url=http://systest.com:8080/auth/realms/master/protocol/openid-connect/certs sasl.oauthbearer.expected.audience=account sasl.oauthbearer.expected.issuer=http://systest.com:8080/auth/realms/master sasl.oauthbearer.sub.claim.name=sub sasl.enabled.mechanisms=GSSAPI,SCRAM-SHA-256,SCRAM-SHA-512 security.inter.broker.protocol=SASL_SSL listeners=SASL_SSL://quasar-hgiovr-1.quasar-hgiovr.root.hwx.site:9093 {code} Based on some extra debugging, it seems that org.apache.kafka.common.network.SaslChannelBuilder#configure calls org.apache.kafka.common.network.SaslChannelBuilder#defaultLoginClass, which selects KerberosLogin (since GSSAPI is listed as enabled), which then gets passed to org.apache.kafka.common.security.authenticator.LoginManager#acquireLoginManager when instantiating the OAuth login manager. Since the serviceName is not defined for the OAuth config (why would it be defined?), the KerberosLogin module crashes. It seems that KAFKA-6246 introduced a bug, which incorrectly uses a single default login class, regardless of the currently configured SASL mechanism. -- This message was sent by Atlassian Jira (v8.20.10#820010)