From 029d9184c6c513e8e3e35814756e7146bf509c04 Mon Sep 17 00:00:00 2001 From: TengYao Chi Date: Sun, 19 Jan 2025 03:14:59 +0800 Subject: [PATCH] KAFKA-18565 Cleanup SaslSetup (#18586) Reviewers: Christo Lolov , Chia-Ping Tsai --- .../kafka/api/CustomQuotaCallbackTest.scala | 2 +- ...gationTokenEndToEndAuthorizationTest.scala | 2 +- .../DescribeAuthorizedOperationsTest.scala | 2 +- .../integration/kafka/api/MetricsTest.scala | 2 +- .../PlaintextEndToEndAuthorizationTest.scala | 2 +- ...aslClientsWithInvalidCredentialsTest.scala | 2 +- .../api/SaslEndToEndAuthorizationTest.scala | 2 +- .../api/SaslMultiMechanismConsumerTest.scala | 2 +- .../api/SaslPlainPlaintextConsumerTest.scala | 2 +- .../kafka/api/SaslPlaintextConsumerTest.scala | 2 +- .../integration/kafka/api/SaslSetup.scala | 23 +++---------------- .../api/SaslSslAdminIntegrationTest.scala | 2 +- .../kafka/api/SaslSslConsumerTest.scala | 2 +- .../kafka/api/SslAdminIntegrationTest.scala | 2 +- .../api/SslEndToEndAuthorizationTest.scala | 2 +- .../integration/kafka/api/UserQuotaTest.scala | 2 +- .../server/GssapiAuthenticationTest.scala | 4 ++-- ...eListenersWithDefaultJaasContextTest.scala | 3 +-- .../server/DelegationTokenRequestsTest.scala | 4 ++-- ...nRequestsWithDisableTokenFeatureTest.scala | 4 ++-- .../server/SaslApiVersionsRequestTest.scala | 4 ++-- ...SaslClientsWithInvalidCredentialsTest.java | 4 ++-- 22 files changed, 29 insertions(+), 47 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala index 565a1f4f28b..4d443a7f9b7 100644 --- a/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala +++ b/core/src/test/scala/integration/kafka/api/CustomQuotaCallbackTest.scala @@ -65,7 +65,7 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some("SCRAM-SHA-256"), KafkaSasl, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Some("SCRAM-SHA-256"), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) this.serverConfig.setProperty(QuotaConfig.CLIENT_QUOTA_CALLBACK_CLASS_CONFIG, classOf[GroupedUserQuotaCallback].getName) this.serverConfig.setProperty(s"${listenerName.configPrefix}${BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG}", classOf[GroupedUserPrincipalBuilder].getName) diff --git a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala index d0798a71412..ab5b587a0e6 100644 --- a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationTest.scala @@ -117,7 +117,7 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism), KafkaSasl)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism))) super.setUp(testInfo) privilegedAdminClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) } diff --git a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala index c61494b122e..0f23b93e31c 100644 --- a/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala +++ b/core/src/test/scala/integration/kafka/api/DescribeAuthorizedOperationsTest.scala @@ -111,7 +111,7 @@ class DescribeAuthorizedOperationsTest extends IntegrationTestHarness with SaslS @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), Both, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) super.setUp(testInfo) TestUtils.waitUntilBrokerMetadataIsPropagated(servers) client = Admin.create(createConfig()) diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala index 71d2764aee8..e08801343fc 100644 --- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala +++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala @@ -65,7 +65,7 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup { } this.consumerConfig.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, "classic") verifyNoRequestMetrics("Request metrics not removed in a previous test") - startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), KafkaSasl, kafkaServerJaasEntryName)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), kafkaServerJaasEntryName)) super.setUp(testInfo) } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala index 0c8bf16b4db..1acd22dc3fa 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala @@ -75,7 +75,7 @@ class PlaintextEndToEndAuthorizationTest extends EndToEndAuthorizationTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(List.empty, None, KafkaSasl)) + startSasl(jaasSections(List.empty, None)) super.setUp(testInfo) } diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index 20435a130e4..0735829a0b1 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -73,7 +73,7 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), Both, + startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) val superuserLoginContext = jaasAdminLoginModule(kafkaClientSaslMechanism) superuserClientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, superuserLoginContext) diff --git a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala index 1929bc87e9a..ec81a98d725 100644 --- a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala @@ -39,7 +39,7 @@ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { // create static config including client login context with credentials for JaasTestUtils 'client2' - startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism), Both)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism))) // set dynamic properties with credentials for JaasTestUtils 'client1' so that dynamic JAAS configuration is also // tested by this set of tests val clientLoginContext = jaasClientLoginModule(kafkaClientSaslMechanism) diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala index 0d562e917ee..30a33c2ab64 100644 --- a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala @@ -34,7 +34,7 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslSetup { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), Both, + startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) super.setUp(testInfo) } diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala index 5da494dc4a7..09f1f5119b1 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala @@ -34,7 +34,7 @@ class SaslPlainPlaintextConsumerTest extends BaseConsumerTest with SaslSetup { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), Both, kafkaServerJaasEntryName)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), kafkaServerJaasEntryName)) super.setUp(testInfo) } diff --git a/core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala index 76bec482f82..36189a57c9f 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlaintextConsumerTest.scala @@ -22,7 +22,7 @@ class SaslPlaintextConsumerTest extends BaseConsumerTest with SaslSetup { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), KafkaSasl, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) super.setUp(testInfo) } diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index 974f9b6cc02..b7d2d920fd9 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -37,15 +37,6 @@ import scala.jdk.CollectionConverters._ import scala.jdk.OptionConverters._ import scala.jdk.javaapi.OptionConverters -/* - * Implements an enumeration for the modes enabled here: - * zk only, kafka only, both, custom KafkaServer. - */ -sealed trait SaslSetupMode -case object ZkSasl extends SaslSetupMode -case object KafkaSasl extends SaslSetupMode -case object Both extends SaslSetupMode - /* * Trait used in SaslTestHarness and EndToEndAuthorizationTest to setup keytab and jaas files. */ @@ -67,11 +58,6 @@ trait SaslSetup { } writeJaasConfigurationToFile(jaasSections) - - val hasZk = jaasSections.exists(_.getModules.asScala.exists(_.name() == "org.apache.zookeeper.server.auth.DigestLoginModule")) - - if (hasZk) - System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider") } protected def initializeKerberos(): Unit = { @@ -93,11 +79,9 @@ trait SaslSetup { } def jaasSections(kafkaServerSaslMechanisms: Seq[String], - kafkaClientSaslMechanism: Option[String], - mode: SaslSetupMode = Both, - kafkaServerEntryName: String = JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME): Seq[JaasSection] = { - val hasKerberos = mode != ZkSasl && - (kafkaServerSaslMechanisms.contains("GSSAPI") || kafkaClientSaslMechanism.contains("GSSAPI")) + kafkaClientSaslMechanism: Option[String], + kafkaServerEntryName: String = JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME): Seq[JaasSection] = { + val hasKerberos = kafkaServerSaslMechanisms.contains("GSSAPI") || kafkaClientSaslMechanism.contains("GSSAPI") if (hasKerberos) maybeCreateEmptyKeytabFiles() Seq(JaasTestUtils.kafkaServerSection(kafkaServerEntryName, kafkaServerSaslMechanisms.asJava, serverKeytabFile.toJava), @@ -117,7 +101,6 @@ trait SaslSetup { // Important if tests leak consumers, producers or brokers LoginManager.closeAll() System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) - System.clearProperty("zookeeper.authProvider.1") Configuration.setConfiguration(null) } diff --git a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala index 3b08e1465f1..06592b9c377 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslAdminIntegrationTest.scala @@ -77,7 +77,7 @@ class SaslSslAdminIntegrationTest extends BaseAdminIntegrationTest with SaslSetu } def setUpSasl(): Unit = { - startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), Both, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) val loginContext = jaasAdminLoginModule("GSSAPI") superuserClientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, loginContext) diff --git a/core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala index 0bfbb81cc63..460ebe2cb4e 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSslConsumerTest.scala @@ -26,7 +26,7 @@ class SaslSslConsumerTest extends BaseConsumerTest with SaslSetup { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), Both, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(Seq("GSSAPI"), Some("GSSAPI"), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) super.setUp(testInfo) } diff --git a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala index 5fcc0449bb5..9e5930d978a 100644 --- a/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslAdminIntegrationTest.scala @@ -127,7 +127,7 @@ class SslAdminIntegrationTest extends SaslSslAdminIntegrationTest { SslAdminIntegrationTest.executor = None SslAdminIntegrationTest.lastUpdateRequestContext = None - startSasl(jaasSections(List.empty, None, KafkaSasl)) + startSasl(jaasSections(List.empty, None)) } override def createConfig: util.Map[String, Object] = { diff --git a/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala index d53d44a6918..3e0ba00d3f9 100644 --- a/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala @@ -75,7 +75,7 @@ class SslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { override val kafkaPrincipal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "server") @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(List.empty, None, KafkaSasl)) + startSasl(jaasSections(List.empty, None)) super.setUp(testInfo) } diff --git a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala index 598444a5deb..3dc04c5f2c1 100644 --- a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala @@ -31,7 +31,7 @@ class UserQuotaTest extends BaseQuotaTest with SaslSetup { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some("GSSAPI"), KafkaSasl, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Some("GSSAPI"), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) super.setUp(testInfo) quotaTestClients.alterClientQuotas( quotaTestClients.clientQuotaAlteration( diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index 670669cb991..667b5523336 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -23,7 +23,7 @@ import java.time.Duration import java.util.{Collections, Properties} import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} import javax.security.auth.login.LoginContext -import kafka.api.{Both, IntegrationTestHarness, SaslSetup} +import kafka.api.{IntegrationTestHarness, SaslSetup} import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.TopicPartition @@ -64,7 +64,7 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { @BeforeEach override def setUp(testInfo: TestInfo): Unit = { TestableKerberosLogin.reset() - startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism), Both)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism))) serverConfig.put(BrokerSecurityConfigs.SSL_CLIENT_AUTH_CONFIG, "required") serverConfig.put(SocketServerConfigs.FAILED_AUTHENTICATION_DELAY_MS_CONFIG, failedAuthenticationDelayMs.toString) super.setUp(testInfo) diff --git a/core/src/test/scala/integration/kafka/server/MultipleListenersWithDefaultJaasContextTest.scala b/core/src/test/scala/integration/kafka/server/MultipleListenersWithDefaultJaasContextTest.scala index e78bb73fe6b..6334b6421a6 100644 --- a/core/src/test/scala/integration/kafka/server/MultipleListenersWithDefaultJaasContextTest.scala +++ b/core/src/test/scala/integration/kafka/server/MultipleListenersWithDefaultJaasContextTest.scala @@ -19,13 +19,12 @@ package kafka.server import java.util.Properties import scala.collection.Seq -import kafka.api.Both import kafka.security.JaasTestUtils.JaasSection class MultipleListenersWithDefaultJaasContextTest extends MultipleListenersWithSameSecurityProtocolBaseTest { override def staticJaasSections: Seq[JaasSection] = - jaasSections(kafkaServerSaslMechanisms.values.flatten.toSeq, Some(kafkaClientSaslMechanism), Both) + jaasSections(kafkaServerSaslMechanisms.values.flatten.toSeq, Some(kafkaClientSaslMechanism)) override protected def dynamicJaasSections: Properties = new Properties diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala index 7e9571db435..2c211eb042a 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import kafka.api.{IntegrationTestHarness, KafkaSasl, SaslSetup} +import kafka.api.{IntegrationTestHarness, SaslSetup} import kafka.security.JaasTestUtils import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig, CreateDelegationTokenOptions, DescribeDelegationTokenOptions} @@ -52,7 +52,7 @@ class DelegationTokenRequestsTest extends IntegrationTestHarness with SaslSetup @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), KafkaSasl, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala index 43694ebb855..c380816f769 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import kafka.api.{KafkaSasl, SaslSetup} +import kafka.api.SaslSetup import kafka.security.JaasTestUtils import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.errors.DelegationTokenDisabledException @@ -42,7 +42,7 @@ class DelegationTokenRequestsWithDisableTokenFeatureTest extends BaseRequestTest @BeforeEach override def setUp(testInfo: TestInfo): Unit = { - startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), KafkaSasl, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + startSasl(jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) super.setUp(testInfo) } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 716b337fc31..70791a4cef0 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import kafka.api.{KafkaSasl, SaslSetup} +import kafka.api.SaslSetup import kafka.security.JaasTestUtils import kafka.server.SaslApiVersionsRequestTest.{kafkaClientSaslMechanism, kafkaServerSaslMechanisms} import org.apache.kafka.common.test.api.{ClusterTemplate, Type, ClusterTestExtensions, ClusterConfig, ClusterInstance} @@ -73,7 +73,7 @@ class SaslApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVe @BeforeEach def setupSasl(): Unit = { sasl = new SaslSetup() {} - sasl.startSasl(sasl.jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), KafkaSasl, JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) + sasl.startSasl(sasl.jaasSections(kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)) } @ClusterTemplate("saslApiVersionsRequestClusterConfig") diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java index 3895f23c88d..cd4198c7c79 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/SaslClientsWithInvalidCredentialsTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.tools.consumer.group; import kafka.api.AbstractSaslTest; -import kafka.api.Both$; import kafka.security.JaasTestUtils; import org.apache.kafka.clients.admin.Admin; @@ -113,7 +112,8 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { @BeforeEach @Override public void setUp(TestInfo testInfo) { - startSasl(jaasSections(KAFKA_SERVER_SASL_MECHANISMS, Some$.MODULE$.apply(KAFKA_CLIENT_SASL_MECHANISM), Both$.MODULE$, + startSasl(jaasSections(KAFKA_SERVER_SASL_MECHANISMS, + Some$.MODULE$.apply(KAFKA_CLIENT_SASL_MECHANISM), JaasTestUtils.KAFKA_SERVER_CONTEXT_NAME)); String superuserLoginContext = jaasAdminLoginModule(KAFKA_CLIENT_SASL_MECHANISM, Option.empty()); this.superuserClientConfig().put(SaslConfigs.SASL_JAAS_CONFIG, superuserLoginContext);