From 57ae6d6706ef3e68953e19c79dddd60ceb76adc9 Mon Sep 17 00:00:00 2001 From: Ming-Yen Chung Date: Mon, 12 May 2025 01:28:30 +0800 Subject: [PATCH] KAFKA-18695 Remove quorum=kraft and kip932 from all integration tests (#19633) Currently, the quorum uses kraft by default, so there's no need to specify it explicitly. For kip932 and isShareGroupTest, they are no longer used after #19542 . Reviewers: PoAn Yang , Ken Huang , Chia-Ping Tsai --- ...EligibleLeaderReplicasIntegrationTest.java | 23 +- .../kafka/admin/RemoteTopicCrudTest.scala | 108 ++-- ...minClientWithPoliciesIntegrationTest.scala | 19 +- .../kafka/api/AuthorizerIntegrationTest.scala | 597 +++++++----------- ...enEndToEndAuthorizationWithOwnerTest.scala | 18 +- .../integration/kafka/api/MetricsTest.scala | 6 +- .../api/PlaintextAdminIntegrationTest.scala | 11 +- .../PlaintextEndToEndAuthorizationTest.scala | 9 +- .../api/ProducerSendWhileDeletionTest.scala | 18 +- ...aslClientsWithInvalidCredentialsTest.scala | 14 +- .../network/DynamicConnectionQuotaTest.scala | 24 +- .../DynamicNumNetworkThreadsTest.scala | 9 +- .../server/GssapiAuthenticationTest.scala | 24 +- .../kafka/server/QuorumTestHarness.scala | 4 - .../scala/kafka/utils/TestInfoUtils.scala | 4 - .../unit/kafka/admin/AddPartitionsTest.scala | 39 +- .../unit/kafka/cluster/PartitionTest.scala | 5 +- .../kafka/integration/MinIsrConfigTest.scala | 8 +- .../unit/kafka/metrics/MetricsTest.scala | 59 +- .../security/authorizer/AuthorizerTest.scala | 148 ++--- .../AddPartitionsToTxnRequestServerTest.scala | 18 +- .../AlterReplicaLogDirsRequestTest.scala | 18 +- ...mCredentialsRequestNotAuthorizedTest.scala | 14 +- ...AlterUserScramCredentialsRequestTest.scala | 42 +- .../server/ControllerMutationQuotaTest.scala | 59 +- .../server/CreateTopicsRequestTest.scala | 28 +- .../CreateTopicsRequestWithPolicyTest.scala | 14 +- ...legationTokenRequestsOnPlainTextTest.scala | 9 +- .../server/DelegationTokenRequestsTest.scala | 9 +- ...nRequestsWithDisableTokenFeatureTest.scala | 9 +- .../server/DeleteRecordsRequestTest.scala | 18 +- .../server/DeleteTopicsRequestTest.scala | 18 +- ...opicsRequestWithDeletionDisabledTest.scala | 9 +- .../server/DescribeClusterRequestTest.scala | 14 +- .../server/DescribeLogDirsRequestTest.scala | 8 +- ...mCredentialsRequestNotAuthorizedTest.scala | 8 +- ...cribeUserScramCredentialsRequestTest.scala | 22 +- .../server/DynamicConfigChangeTest.scala | 92 ++- .../kafka/server/EdgeCaseRequestTest.scala | 28 +- .../server/FetchRequestMaxBytesTest.scala | 9 +- .../unit/kafka/server/FetchRequestTest.scala | 68 +- ...aMetricReporterExceptionHandlingTest.scala | 9 +- .../server/KafkaMetricsReporterTest.scala | 9 +- .../kafka/server/ListOffsetsRequestTest.scala | 23 +- .../unit/kafka/server/LogOffsetTest.scala | 44 +- .../unit/kafka/server/LogRecoveryTest.scala | 24 +- .../kafka/server/MetadataRequestTest.scala | 69 +- .../OffsetsForLeaderEpochRequestTest.scala | 13 +- .../kafka/server/ProduceRequestTest.scala | 22 +- .../unit/kafka/server/ReplicaFetchTest.scala | 9 +- .../kafka/server/ReplicationQuotasTest.scala | 19 +- .../unit/kafka/server/RequestQuotaTest.scala | 34 +- .../kafka/server/ServerShutdownTest.scala | 24 +- .../epoch/LeaderEpochIntegrationTest.scala | 19 +- .../group/AuthorizerIntegrationTest.java | 8 +- 55 files changed, 776 insertions(+), 1211 deletions(-) diff --git a/core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java b/core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java index 65c75d584db..28c12cf6bce 100644 --- a/core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java +++ b/core/src/test/java/kafka/server/integration/EligibleLeaderReplicasIntegrationTest.java @@ -50,9 +50,8 @@ import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; import java.io.File; import java.time.Duration; @@ -137,9 +136,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes if (adminClient != null) adminClient.close(); } - @ParameterizedTest - @ValueSource(strings = {"kraft"}) - public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr(String quorum) throws ExecutionException, InterruptedException { + @Test + public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr() throws ExecutionException, InterruptedException { adminClient.createTopics( List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); @@ -224,9 +222,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes ); } - @ParameterizedTest - @ValueSource(strings = {"kraft"}) - public void testElrMemberCanBeElected(String quorum) throws ExecutionException, InterruptedException { + @Test + public void testElrMemberCanBeElected() throws ExecutionException, InterruptedException { adminClient.createTopics( List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); @@ -300,9 +297,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes } } - @ParameterizedTest - @ValueSource(strings = {"kraft"}) - public void testElrMemberShouldBeKickOutWhenUncleanShutdown(String quorum) throws ExecutionException, InterruptedException { + @Test + public void testElrMemberShouldBeKickOutWhenUncleanShutdown() throws ExecutionException, InterruptedException { adminClient.createTopics( List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); @@ -361,9 +357,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes /* This test is only valid for KIP-966 part 1. When the unclean recovery is implemented, it should be removed. */ - @ParameterizedTest - @ValueSource(strings = {"kraft"}) - public void testLastKnownLeaderShouldBeElectedIfEmptyElr(String quorum) throws ExecutionException, InterruptedException { + @Test + public void testLastKnownLeaderShouldBeElectedIfEmptyElr() throws ExecutionException, InterruptedException { adminClient.createTopics( List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get(); TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000); diff --git a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala index 4f66dd9e311..de1a2c88f2a 100644 --- a/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala +++ b/core/src/test/scala/integration/kafka/admin/RemoteTopicCrudTest.scala @@ -28,9 +28,9 @@ import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentState} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.function.Executable -import org.junit.jupiter.api.{BeforeEach, Tag, TestInfo} +import org.junit.jupiter.api.{BeforeEach, Tag, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{CsvSource, ValueSource} +import org.junit.jupiter.params.provider.CsvSource import java.util import java.util.concurrent.atomic.AtomicInteger @@ -73,9 +73,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { testTopicName = s"${info.getTestMethod.get().getName}-${Random.alphanumeric.take(10).mkString}" } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateRemoteTopicWithValidRetentionTime(quorum: String): Unit = { + @Test + def testCreateRemoteTopicWithValidRetentionTime(): Unit = { val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200") @@ -85,9 +84,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(topicConfig) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateRemoteTopicWithValidRetentionSize(quorum: String): Unit = { + @Test + def testCreateRemoteTopicWithValidRetentionSize(): Unit = { val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512") @@ -97,9 +95,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(topicConfig) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateRemoteTopicWithInheritedLocalRetentionTime(quorum: String): Unit = { + @Test + def testCreateRemoteTopicWithInheritedLocalRetentionTime(): Unit = { // inherited local retention ms is 1000 val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -109,9 +106,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(topicConfig) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateRemoteTopicWithInheritedLocalRetentionSize(quorum: String): Unit = { + @Test + def testCreateRemoteTopicWithInheritedLocalRetentionSize(): Unit = { // inherited local retention bytes is 1024 val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -121,9 +117,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(topicConfig) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateRemoteTopicWithInvalidRetentionTime(quorum: String): Unit = { + @Test + def testCreateRemoteTopicWithInvalidRetentionTime(): Unit = { // inherited local retention ms is 1000 val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -133,9 +128,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { topicConfig = topicConfig)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateRemoteTopicWithInvalidRetentionSize(quorum: String): Unit = { + @Test + def testCreateRemoteTopicWithInvalidRetentionSize(): Unit = { // inherited local retention bytes is 1024 val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -145,9 +139,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { topicConfig = topicConfig)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateCompactedRemoteStorage(quorum: String): Unit = { + @Test + def testCreateCompactedRemoteStorage(): Unit = { val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact") @@ -158,8 +151,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { // `remote.log.delete.on.disable` and `remote.log.copy.disable` only works in KRaft mode. @ParameterizedTest - @CsvSource(Array("kraft,true,true", "kraft,true,false", "kraft,false,true", "kraft,false,false")) - def testCreateRemoteTopicWithCopyDisabledAndDeleteOnDisable(quorum: String, copyDisabled: Boolean, deleteOnDisable: Boolean): Unit = { + @CsvSource(Array("true,true", "true,false", "false,true", "false,false")) + def testCreateRemoteTopicWithCopyDisabledAndDeleteOnDisable(copyDisabled: Boolean, deleteOnDisable: Boolean): Unit = { val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, copyDisabled.toString) topicConfig.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString) @@ -169,9 +162,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { } // `remote.log.delete.on.disable` only works in KRaft mode. - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateTopicRetentionMsValidationWithRemoteCopyDisabled(quorum: String): Unit = { + @Test + def testCreateTopicRetentionMsValidationWithRemoteCopyDisabled(): Unit = { val testTopicName2 = testTopicName + "2" val testTopicName3 = testTopicName + "3" val errorMsgMs = "When `remote.log.copy.disable` is set to true, the `local.retention.ms` and `retention.ms` " + @@ -235,9 +227,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { admin.incrementalAlterConfigs(configs).all().get() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateTopicRetentionBytesValidationWithRemoteCopyDisabled(quorum: String): Unit = { + @Test + def testCreateTopicRetentionBytesValidationWithRemoteCopyDisabled(): Unit = { val testTopicName2 = testTopicName + "2" val testTopicName3 = testTopicName + "3" val errorMsgBytes = "When `remote.log.copy.disable` is set to true, the `local.retention.bytes` and `retention.bytes` " + @@ -300,9 +291,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { admin.incrementalAlterConfigs(configs).all().get() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testEnableRemoteLogOnExistingTopicTest(quorum: String): Unit = { + @Test + def testEnableRemoteLogOnExistingTopicTest(): Unit = { val admin = createAdminClient() val topicConfig = new Properties() TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor, @@ -318,9 +308,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(topicConfig) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testEnableRemoteLogWhenSystemRemoteStorageIsDisabled(quorum: String): Unit = { + @Test + def testEnableRemoteLogWhenSystemRemoteStorageIsDisabled(): Unit = { val admin = createAdminClient() val topicConfigWithRemoteStorage = new Properties() @@ -342,9 +331,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { assertTrue(errorMessage.getMessage.contains("Tiered Storage functionality is disabled in the broker")) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUpdateTopicConfigWithValidRetentionTimeTest(quorum: String): Unit = { + @Test + def testUpdateTopicConfigWithValidRetentionTimeTest(): Unit = { val admin = createAdminClient() val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -363,9 +351,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(topicConfig) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUpdateTopicConfigWithValidRetentionSizeTest(quorum: String): Unit = { + @Test + def testUpdateTopicConfigWithValidRetentionSizeTest(): Unit = { val admin = createAdminClient() val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -384,9 +371,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(topicConfig) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUpdateTopicConfigWithInheritedLocalRetentionTime(quorum: String): Unit = { + @Test + def testUpdateTopicConfigWithInheritedLocalRetentionTime(): Unit = { val admin = createAdminClient() val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -404,9 +390,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { () => admin.incrementalAlterConfigs(configs).all().get()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUpdateTopicConfigWithInheritedLocalRetentionSize(quorum: String): Unit = { + @Test + def testUpdateTopicConfigWithInheritedLocalRetentionSize(): Unit = { val admin = createAdminClient() val topicConfig = new Properties() topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -425,9 +410,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { } // The remote storage config validation on controller level only works in KRaft - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUpdateTopicConfigWithDisablingRemoteStorage(quorum: String): Unit = { + @Test + def testUpdateTopicConfigWithDisablingRemoteStorage(): Unit = { val admin = createAdminClient() val topicConfig = new Properties topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -446,9 +430,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { "If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUpdateTopicConfigWithDisablingRemoteStorageWithDeleteOnDisable(quorum: String): Unit = { + @Test + def testUpdateTopicConfigWithDisablingRemoteStorageWithDeleteOnDisable(): Unit = { val admin = createAdminClient() val topicConfig = new Properties topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -473,9 +456,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { verifyRemoteLogTopicConfigs(newProps) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTopicDeletion(quorum: String): Unit = { + @Test + def testTopicDeletion(): Unit = { MyRemoteStorageManager.deleteSegmentEventCounter.set(0) val numPartitions = 2 val topicConfig = new Properties() @@ -492,9 +474,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { "Remote log segments should be deleted only once by the leader") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic(quorum: String): Unit = { + @Test + def testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic(): Unit = { val topicConfig = new Properties() topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true") @@ -510,9 +491,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness { faultHandler.setIgnore(true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled(quorum: String): Unit = { + @Test + def testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled(): Unit = { val topicConfig = new Properties() topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, false.toString) diff --git a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala index 3c73bf13f8b..d8e75053068 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientWithPoliciesIntegrationTest.scala @@ -29,9 +29,7 @@ import org.apache.kafka.server.policy.AlterConfigPolicy import org.apache.kafka.storage.internals.log.LogConfig import org.apache.kafka.test.TestUtils.assertFutureThrows import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue} -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout} import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -79,9 +77,8 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with props.put(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[Policy]) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testValidAlterConfigs(quorum: String): Unit = { + @Test + def testValidAlterConfigs(): Unit = { client = Admin.create(createConfig) // Create topics val topic1 = "describe-alter-configs-topic-1" @@ -100,16 +97,14 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with PlaintextAdminIntegrationTest.checkValidAlterConfigs(client, this, topicResource1, topicResource2, maxMessageBytes, retentionMs) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testInvalidAlterConfigs(quorum: String): Unit = { + @Test + def testInvalidAlterConfigs(): Unit = { client = Admin.create(createConfig) PlaintextAdminIntegrationTest.checkInvalidAlterConfigs(this, client) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testInvalidAlterConfigsDueToPolicy(quorum: String): Unit = { + @Test + def testInvalidAlterConfigsDueToPolicy(): Unit = { client = Admin.create(createConfig) // Create topics diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index c0325a4048b..9d497431a68 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -53,7 +53,7 @@ import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{CsvSource, MethodSource, ValueSource} +import org.junit.jupiter.params.provider.{CsvSource, MethodSource} import java.util.Collections.singletonList import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic @@ -833,9 +833,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testAuthorizationWithTopicExisting(quorum: String): Unit = { + @Test + def testAuthorizationWithTopicExisting(): Unit = { //First create the topic so we have a valid topic ID sendRequests(mutable.Map(ApiKeys.CREATE_TOPICS -> createTopicsRequest)) @@ -895,9 +894,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAuthorizationWithTopicNotExisting(quorum: String): Unit = { + @Test + def testAuthorizationWithTopicNotExisting(): Unit = { val id = Uuid.randomUuid() val topicNames = Map(id -> "topic") val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest]( @@ -976,9 +974,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAuthorizationFetchV12WithTopicNotExisting(quorum: String): Unit = { + @Test + def testAuthorizationFetchV12WithTopicNotExisting(): Unit = { val id = Uuid.ZERO_UUID val topicNames = Map(id -> "topic") val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest]( @@ -988,9 +985,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequests(requestKeyToRequest, false, topicNames) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateTopicAuthorizationWithClusterCreate(quorum: String): Unit = { + @Test + def testCreateTopicAuthorizationWithClusterCreate(): Unit = { removeAllClientAcls() val resources = Set[ResourceType](TOPIC) @@ -1001,9 +997,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(createTopicsRequest, resources, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchFollowerRequest(quorum: String): Unit = { + @Test + def testFetchFollowerRequest(): Unit = { createTopicWithBrokerPrincipal(topic) val request = createFetchFollowerRequest @@ -1049,9 +1044,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { succeededPartitionDatas.foreach(partitionData => assertEquals(MemoryRecords.EMPTY, partitionData.records)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(quorum: String): Unit = { + @Test + def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(): Unit = { createTopicWithBrokerPrincipal(topic) val data = new IncrementalAlterConfigsRequestData @@ -1073,9 +1067,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testOffsetsForLeaderEpochClusterPermission(quorum: String): Unit = { + @Test + def testOffsetsForLeaderEpochClusterPermission(): Unit = { createTopicWithBrokerPrincipal(topic) val request = offsetsForLeaderEpochRequest @@ -1092,50 +1085,44 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testProduceWithNoTopicAccess(quorum: String): Unit = { + @Test + def testProduceWithNoTopicAccess(): Unit = { createTopicWithBrokerPrincipal(topic) val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testProduceWithTopicDescribe(quorum: String): Unit = { + @Test + def testProduceWithTopicDescribe(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testProduceWithTopicRead(quorum: String): Unit = { + @Test + def testProduceWithTopicRead(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testProduceWithTopicWrite(quorum: String): Unit = { + @Test + def testProduceWithTopicWrite(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) val producer = createProducer() sendRecords(producer, numRecords, tp) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreatePermissionOnTopicToWriteToNonExistentTopic(quorum: String): Unit = { + @Test + def testCreatePermissionOnTopicToWriteToNonExistentTopic(): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(TOPIC) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreatePermissionOnClusterToWriteToNonExistentTopic(quorum: String): Unit = { + @Test + def testCreatePermissionOnClusterToWriteToNonExistentTopic(): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(CLUSTER) } @@ -1483,9 +1470,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }, "Partition metadata not propagated.") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreatePermissionMetadataRequestAutoCreate(quorum: String): Unit = { + @Test + def testCreatePermissionMetadataRequestAutoCreate(): Unit = { val readAcls = topicReadAcl(topicResource) addAndVerifyAcls(readAcls, topicResource) brokers.foreach(b => assertEquals(Optional.empty, b.metadataCache.getLeaderAndIsr(topic, 0))) @@ -2046,17 +2032,15 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.endOffsets(Set(tp).asJava) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupApiWithNoGroupAcl(quorum: String): Unit = { + @Test + def testDescribeGroupApiWithNoGroupAcl(): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val result = createAdminClient().describeConsumerGroups(Seq(group).asJava) JTestUtils.assertFutureThrows(classOf[GroupAuthorizationException], result.describedGroups().get(group)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupApiWithGroupDescribe(quorum: String): Unit = { + @Test + def testDescribeGroupApiWithGroupDescribe(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) @@ -2141,9 +2125,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { JTestUtils.assertFutureThrows(classOf[GroupAuthorizationException], result.deletedGroups().get(group)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteGroupApiWithNoDeleteGroupAcl2(quorum: String): Unit = { + @Test + def testDeleteGroupApiWithNoDeleteGroupAcl2(): Unit = { val result = createAdminClient().deleteConsumerGroups(Seq(group).asJava) JTestUtils.assertFutureThrows(classOf[GroupAuthorizationException], result.deletedGroups().get(group)) } @@ -2200,16 +2183,14 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { JTestUtils.assertFutureThrows(classOf[TopicAuthorizationException], result.partitionResult(tp)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteGroupOffsetsWithNoAcl(quorum: String): Unit = { + @Test + def testDeleteGroupOffsetsWithNoAcl(): Unit = { val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) JTestUtils.assertFutureThrows(classOf[GroupAuthorizationException], result.all()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterGroupConfigsWithAlterAcl(quorum: String): Unit = { + @Test + def testIncrementalAlterGroupConfigsWithAlterAcl(): Unit = { addAndVerifyAcls(groupAlterConfigsAcl(groupResource), groupResource) val request = incrementalAlterGroupConfigsRequest @@ -2217,9 +2198,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterGroupConfigsWithOperationAll(quorum: String): Unit = { + @Test + def testIncrementalAlterGroupConfigsWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2228,9 +2208,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterGroupConfigsWithoutAlterAcl(quorum: String): Unit = { + @Test + def testIncrementalAlterGroupConfigsWithoutAlterAcl(): Unit = { removeAllClientAcls() val request = incrementalAlterGroupConfigsRequest @@ -2238,9 +2217,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupConfigsWithDescribeAcl(quorum: String): Unit = { + @Test + def testDescribeGroupConfigsWithDescribeAcl(): Unit = { addAndVerifyAcls(groupDescribeConfigsAcl(groupResource), groupResource) val request = describeGroupConfigsRequest @@ -2248,9 +2226,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupConfigsWithOperationAll(quorum: String): Unit = { + @Test + def testDescribeGroupConfigsWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2259,9 +2236,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupConfigsWithoutDescribeAcl(quorum: String): Unit = { + @Test + def testDescribeGroupConfigsWithoutDescribeAcl(): Unit = { removeAllClientAcls() val request = describeGroupConfigsRequest @@ -2269,42 +2245,37 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnauthorizedDeleteTopicsWithoutDescribe(quorum: String): Unit = { + @Test + def testUnauthorizedDeleteTopicsWithoutDescribe(): Unit = { val deleteResponse = connectAndReceive[DeleteTopicsResponse](deleteTopicsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnauthorizedDeleteTopicsWithDescribe(quorum: String): Unit = { + @Test + def testUnauthorizedDeleteTopicsWithDescribe(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val deleteResponse = connectAndReceive[DeleteTopicsResponse](deleteTopicsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteTopicsWithWildCardAuth(quorum: String): Unit = { + @Test + def testDeleteTopicsWithWildCardAuth(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)), new ResourcePattern(TOPIC, "*", LITERAL)) val deleteResponse = connectAndReceive[DeleteTopicsResponse](deleteTopicsRequest) assertEquals(Errors.NONE.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnauthorizedDeleteRecordsWithoutDescribe(quorum: String): Unit = { + @Test + def testUnauthorizedDeleteRecordsWithoutDescribe(): Unit = { val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteRecordsResponse.data.topics.asScala.head. partitions.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnauthorizedDeleteRecordsWithDescribe(quorum: String): Unit = { + @Test + def testUnauthorizedDeleteRecordsWithDescribe(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) @@ -2312,9 +2283,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { partitions.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteRecordsWithWildCardAuth(quorum: String): Unit = { + @Test + def testDeleteRecordsWithWildCardAuth(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)), new ResourcePattern(TOPIC, "*", LITERAL)) val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) @@ -2322,40 +2292,35 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { partitions.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnauthorizedCreatePartitions(quorum: String): Unit = { + @Test + def testUnauthorizedCreatePartitions(): Unit = { val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, createPartitionsResponse.data.results.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreatePartitionsWithWildCardAuth(quorum: String): Unit = { + @Test + def testCreatePartitionsWithWildCardAuth(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALTER, ALLOW)), new ResourcePattern(TOPIC, "*", LITERAL)) val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest) assertEquals(Errors.NONE.code, createPartitionsResponse.data.results.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(quorum: String): Unit = { + @Test + def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), transactionalIdResource) val producer = buildTransactionalProducer() assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(quorum: String): Unit = { + @Test + def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(): Unit = { val producer = buildTransactionalProducer() assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSendOffsetsWithNoConsumerGroupDescribeAccess(quorum: String): Unit = { + @Test + def testSendOffsetsWithNoConsumerGroupDescribeAccess(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CLUSTER_ACTION, ALLOW)), clusterResource) @@ -2369,9 +2334,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSendOffsetsWithNoConsumerGroupWriteAccess(quorum: String): Unit = { + @Test + def testSendOffsetsWithNoConsumerGroupWriteAccess(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2384,9 +2348,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(quorum: String): Unit = { + @Test + def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) assertIdempotentSendAuthorizationFailure() @@ -2423,9 +2386,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertClusterAuthFailure() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIdempotentProducerNoIdempotentWriteAclInProduce(quorum: String): Unit = { + @Test + def testIdempotentProducerNoIdempotentWriteAclInProduce(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, IDEMPOTENT_WRITE, ALLOW)), clusterResource) @@ -2452,17 +2414,15 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertTrue(e.getCause.isInstanceOf[TopicAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldInitTransactionsWhenAclSet(quorum: String): Unit = { + @Test + def shouldInitTransactionsWhenAclSet(): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) val producer = buildTransactionalProducer() producer.initTransactions() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(quorum: String): Unit = { + @Test + def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2477,9 +2437,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Set(topic), e.unauthorizedTopics.asScala) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTransactionalProducerTopicAuthorizationExceptionInCommit(quorum: String): Unit = { + @Test + def testTransactionalProducerTopicAuthorizationExceptionInCommit(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2495,9 +2454,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(quorum: String): Unit = { + @Test + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2510,9 +2468,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { JTestUtils.assertFutureThrows(classOf[TransactionalIdAuthorizationException], future) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(quorum: String): Unit = { + @Test + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2525,9 +2482,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.commitTransaction()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testListTransactionsAuthorization(quorum: String): Unit = { + @Test + def testListTransactionsAuthorization(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2559,9 +2515,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertListTransactionResult(expectedTransactionalIds = Set(transactionalId)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldNotIncludeUnauthorizedTopicsInDescribeTransactionsResponse(quorum: String): Unit = { + @Test + def shouldNotIncludeUnauthorizedTopicsInDescribeTransactionsResponse(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2582,9 +2537,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(List.empty, transactionStateData.topics.asScala.toList) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(quorum: String): Unit = { + @Test + def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2602,9 +2556,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { producer.abortTransaction() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(quorum: String): Unit = { + @Test + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), groupResource) val producer = buildTransactionalProducer() @@ -2620,9 +2573,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(quorum: String): Unit = { + @Test + def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, IDEMPOTENT_WRITE, ALLOW)), clusterResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2631,18 +2583,16 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } // Verify that metadata request without topics works without any ACLs and returns cluster id - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testClusterId(quorum: String): Unit = { + @Test + def testClusterId(): Unit = { val request = new requests.MetadataRequest.Builder(List.empty.asJava, false).build() val response = connectAndReceive[MetadataResponse](request) assertEquals(Collections.emptyMap, response.errorCounts) assertFalse(response.clusterId.isEmpty, "Cluster id not returned") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testRetryProducerInitializationAfterPermissionFix(quorum: String): Unit = { + @Test + def testRetryProducerInitializationAfterPermissionFix(): Unit = { createTopicWithBrokerPrincipal(topic) val wildcard = new ResourcePattern(TOPIC, ResourcePattern.WILDCARD_RESOURCE, LITERAL) val prefixed = new ResourcePattern(TOPIC, "t", PREFIXED) @@ -2664,9 +2614,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { producer.close() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAuthorizeByResourceTypeMultipleAddAndRemove(quorum: String): Unit = { + @Test + def testAuthorizeByResourceTypeMultipleAddAndRemove(): Unit = { createTopicWithBrokerPrincipal(topic) for (_ <- 1 to 3) { @@ -2682,9 +2631,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAuthorizeByResourceTypeIsolationUnrelatedDenyWontDominateAllow(quorum: String): Unit = { + @Test + def testAuthorizeByResourceTypeIsolationUnrelatedDenyWontDominateAllow(): Unit = { createTopicWithBrokerPrincipal(topic) createTopicWithBrokerPrincipal("topic-2") createTopicWithBrokerPrincipal("to") @@ -2705,9 +2653,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendSuccess() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAuthorizeByResourceTypeDenyTakesPrecedence(quorum: String): Unit = { + @Test + def testAuthorizeByResourceTypeDenyTakesPrecedence(): Unit = { createTopicWithBrokerPrincipal(topic) val allowWriteAce = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW) addAndVerifyAcls(Set(allowWriteAce), topicResource) @@ -2718,9 +2665,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAuthorizeByResourceTypeWildcardResourceDenyDominate(quorum: String): Unit = { + @Test + def testAuthorizeByResourceTypeWildcardResourceDenyDominate(): Unit = { createTopicWithBrokerPrincipal(topic) val wildcard = new ResourcePattern(TOPIC, ResourcePattern.WILDCARD_RESOURCE, LITERAL) val prefixed = new ResourcePattern(TOPIC, "t", PREFIXED) @@ -2736,9 +2682,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAuthorizeByResourceTypePrefixedResourceDenyDominate(quorum: String): Unit = { + @Test + def testAuthorizeByResourceTypePrefixedResourceDenyDominate(): Unit = { createTopicWithBrokerPrincipal(topic) val prefixed = new ResourcePattern(TOPIC, topic.substring(0, 1), PREFIXED) val literal = new ResourcePattern(TOPIC, topic, LITERAL) @@ -2750,9 +2695,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { + @Test + def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(): Unit = { removeAllClientAcls() // MetadataRequest versions older than 1 are not supported. @@ -2761,9 +2705,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMetadataClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { + @Test + def testMetadataClusterAuthorizedOperationsWithDescribeAndAlterCluster(): Unit = { removeAllClientAcls() val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) @@ -2782,9 +2725,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeTopicAclWithOperationAll(quorum: String): Unit = { + @Test + def testDescribeTopicAclWithOperationAll(): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2807,9 +2749,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Errors.NONE, topicResponse.error) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeTopicConfigsAclWithOperationAll(quorum: String): Unit = { + @Test + def testDescribeTopicConfigsAclWithOperationAll(): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2847,9 +2788,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { + @Test + def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(): Unit = { removeAllClientAcls() for (version <- ApiKeys.DESCRIBE_CLUSTER.oldestVersion to ApiKeys.DESCRIBE_CLUSTER.latestVersion) { @@ -2857,9 +2797,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeClusterClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { + @Test + def testDescribeClusterClusterAuthorizedOperationsWithDescribeAndAlterCluster(): Unit = { removeAllClientAcls() val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) @@ -2877,9 +2816,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testHostAddressBasedAcls(quorum: String): Unit = { + @Test + def testHostAddressBasedAcls(): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2919,9 +2857,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertDoesNotThrow(closeConsumer, "Exception not expected on closing consumer") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithGroupReadAndTopicDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithGroupReadAndTopicDescribeAcl(): Unit = { addAndVerifyAcls(groupReadAcl(groupResource), groupResource) addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) @@ -2930,9 +2867,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithOperationAll(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) @@ -2942,9 +2878,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithoutGroupReadOrTopicDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithoutGroupReadOrTopicDescribeAcl(): Unit = { removeAllClientAcls() val request = consumerGroupHeartbeatRequest @@ -2952,9 +2887,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithoutGroupReadAcl(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithoutGroupReadAcl(): Unit = { addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) val request = consumerGroupHeartbeatRequest @@ -2963,9 +2897,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithoutTopicDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithoutTopicDescribeAcl(): Unit = { addAndVerifyAcls(groupReadAcl(groupResource), groupResource) val request = consumerGroupHeartbeatRequest @@ -2974,9 +2907,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithRegex(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithRegex(): Unit = { createTopicWithBrokerPrincipal(topic) val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2986,9 +2918,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendAndReceiveRegexHeartbeat(response, listenerName, Some(1)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithRegexWithoutTopicDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithRegexWithoutTopicDescribeAcl(): Unit = { createTopicWithBrokerPrincipal(topic) val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2997,9 +2928,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendAndReceiveRegexHeartbeat(response, listenerName, None) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithRegexWithDifferentMemberAcls(quorum: String): Unit = { + @Test + def testConsumerGroupHeartbeatWithRegexWithDifferentMemberAcls(): Unit = { createTopicWithBrokerPrincipal(topic, numPartitions = 2) val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -3035,9 +2965,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendAndReceiveRegexHeartbeat(member1Response, interBrokerListenerName, Some(0), fullRequest = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithGroupReadAndTopicDescribeAcl(quorum: String): Unit = { + @Test + def testShareGroupHeartbeatWithGroupReadAndTopicDescribeAcl(): Unit = { addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) @@ -3046,9 +2975,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithOperationAll(quorum: String): Unit = { + @Test + def testShareGroupHeartbeatWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) @@ -3058,9 +2986,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithoutGroupReadOrTopicDescribeAcl(quorum: String): Unit = { + @Test + def testShareGroupHeartbeatWithoutGroupReadOrTopicDescribeAcl(): Unit = { removeAllClientAcls() val request = shareGroupHeartbeatRequest @@ -3068,9 +2995,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithoutGroupReadAcl(quorum: String): Unit = { + @Test + def testShareGroupHeartbeatWithoutGroupReadAcl(): Unit = { addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) val request = shareGroupHeartbeatRequest @@ -3078,9 +3004,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupHeartbeatWithoutTopicDescribeAcl(quorum: String): Unit = { + @Test + def testShareGroupHeartbeatWithoutTopicDescribeAcl(): Unit = { addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) val request = shareGroupHeartbeatRequest @@ -3099,9 +3024,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { removeAllClientAcls() } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithGroupDescribeAndTopicDescribeAcl(quorum: String): Unit = { + @Test + def testShareGroupDescribeWithGroupDescribeAndTopicDescribeAcl(): Unit = { createShareGroupToDescribe() addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) @@ -3111,9 +3035,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithOperationAll(quorum: String): Unit = { + @Test + def testShareGroupDescribeWithOperationAll(): Unit = { createShareGroupToDescribe() val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) @@ -3125,9 +3048,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithoutGroupDescribeAcl(quorum: String): Unit = { + @Test + def testShareGroupDescribeWithoutGroupDescribeAcl(): Unit = { createShareGroupToDescribe() addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) @@ -3136,9 +3058,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(quorum: String): Unit = { + @Test + def testShareGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(): Unit = { createShareGroupToDescribe() val request = shareGroupDescribeRequest @@ -3146,9 +3067,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithGroupReadAndTopicReadAcl(quorum: String): Unit = { + @Test + def testShareFetchWithGroupReadAndTopicReadAcl(): Unit = { addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) addAndVerifyAcls(topicReadAcl(topicResource), topicResource) @@ -3157,9 +3077,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithOperationAll(quorum: String): Unit = { + @Test + def testShareFetchWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) @@ -3169,9 +3088,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithoutGroupReadOrTopicReadAcl(quorum: String): Unit = { + @Test + def testShareFetchWithoutGroupReadOrTopicReadAcl(): Unit = { removeAllClientAcls() val request = createShareFetchRequest @@ -3179,9 +3097,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithoutGroupReadAcl(quorum: String): Unit = { + @Test + def testShareFetchWithoutGroupReadAcl(): Unit = { addAndVerifyAcls(topicReadAcl(topicResource), topicResource) val request = createShareFetchRequest @@ -3189,9 +3106,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareFetchWithoutTopicReadAcl(quorum: String): Unit = { + @Test + def testShareFetchWithoutTopicReadAcl(): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) @@ -3200,9 +3116,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, Errors.forCode(response.data.responses.get(0).partitions.get(0).errorCode)) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeWithGroupReadAndTopicReadAcl(quorum: String): Unit = { + @Test + def testShareAcknowledgeWithGroupReadAndTopicReadAcl(): Unit = { addAndVerifyAcls(shareGroupReadAcl(shareGroupResource), shareGroupResource) addAndVerifyAcls(topicReadAcl(topicResource), topicResource) @@ -3211,9 +3126,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeWithOperationAll(quorum: String): Unit = { + @Test + def testShareAcknowledgeWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) @@ -3223,9 +3137,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeWithoutGroupReadOrTopicReadAcl(quorum: String): Unit = { + @Test + def testShareAcknowledgeWithoutGroupReadOrTopicReadAcl(): Unit = { removeAllClientAcls() val request = shareAcknowledgeRequest @@ -3233,9 +3146,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareAcknowledgeFetchWithoutGroupReadAcl(quorum: String): Unit = { + @Test + def testShareAcknowledgeFetchWithoutGroupReadAcl(): Unit = { addAndVerifyAcls(topicReadAcl(topicResource), topicResource) val request = shareAcknowledgeRequest @@ -3243,9 +3155,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testInitializeShareGroupStateWithClusterAcl(quorum: String): Unit = { + @Test + def testInitializeShareGroupStateWithClusterAcl(): Unit = { addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) val request = initializeShareGroupStateRequest @@ -3253,9 +3164,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testInitializeShareGroupStateWithOperationAll(quorum: String): Unit = { + @Test + def testInitializeShareGroupStateWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) @@ -3264,9 +3174,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testInitializeShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + @Test + def testInitializeShareGroupStateWithoutClusterAcl(): Unit = { removeAllClientAcls() val request = initializeShareGroupStateRequest @@ -3274,9 +3183,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateWithClusterAcl(quorum: String): Unit = { + @Test + def testReadShareGroupStateWithClusterAcl(): Unit = { addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) val request = readShareGroupStateRequest @@ -3284,9 +3192,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateWithOperationAll(quorum: String): Unit = { + @Test + def testReadShareGroupStateWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) @@ -3295,9 +3202,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + @Test + def testReadShareGroupStateWithoutClusterAcl(): Unit = { removeAllClientAcls() val request = readShareGroupStateRequest @@ -3305,9 +3211,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testWriteShareGroupStateWithClusterAcl(quorum: String): Unit = { + @Test + def testWriteShareGroupStateWithClusterAcl(): Unit = { addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) val request = writeShareGroupStateRequest @@ -3315,9 +3220,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testWriteShareGroupStateWithOperationAll(quorum: String): Unit = { + @Test + def testWriteShareGroupStateWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) @@ -3326,9 +3230,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testWriteShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + @Test + def testWriteShareGroupStateWithoutClusterAcl(): Unit = { removeAllClientAcls() val request = writeShareGroupStateRequest @@ -3336,9 +3239,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupStateWithClusterAcl(quorum: String): Unit = { + @Test + def testDeleteShareGroupStateWithClusterAcl(): Unit = { addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) val request = deleteShareGroupStateRequest @@ -3346,9 +3248,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupStateWithOperationAll(quorum: String): Unit = { + @Test + def testDeleteShareGroupStateWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) @@ -3357,9 +3258,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupStateWithoutClusterAcl(quorum: String): Unit = { + @Test + def testDeleteShareGroupStateWithoutClusterAcl(): Unit = { removeAllClientAcls() val request = deleteShareGroupStateRequest @@ -3367,9 +3267,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateSummaryWithClusterAcl(quorum: String): Unit = { + @Test + def testReadShareGroupStateSummaryWithClusterAcl(): Unit = { addAndVerifyAcls(clusterAcl(clusterResource), clusterResource) val request = readShareGroupStateSummaryRequest @@ -3377,9 +3276,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateSummaryWithOperationAll(quorum: String): Unit = { + @Test + def testReadShareGroupStateSummaryWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), clusterResource) @@ -3388,9 +3286,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testReadShareGroupStateSummaryWithoutClusterAcl(quorum: String): Unit = { + @Test + def testReadShareGroupStateSummaryWithoutClusterAcl(): Unit = { removeAllClientAcls() val request = readShareGroupStateRequest @@ -3398,9 +3295,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithGroupDescribeAndTopicDescribeAcl(quorum: String): Unit = { + @Test + def testDescribeShareGroupOffsetsWithGroupDescribeAndTopicDescribeAcl(): Unit = { addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) @@ -3409,9 +3305,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithOperationAll(quorum: String): Unit = { + @Test + def testDescribeShareGroupOffsetsWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) @@ -3421,9 +3316,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithoutGroupDescribeOrTopicDescribeAcl(quorum: String): Unit = { + @Test + def testDescribeShareGroupOffsetsWithoutGroupDescribeOrTopicDescribeAcl(): Unit = { removeAllClientAcls() val request = describeShareGroupOffsetsRequest @@ -3431,9 +3325,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithoutGroupDescribeAcl(quorum: String): Unit = { + @Test + def testDescribeShareGroupOffsetsWithoutGroupDescribeAcl(): Unit = { addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) val request = describeShareGroupOffsetsRequest @@ -3441,9 +3334,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDescribeShareGroupOffsetsWithoutTopicDescribeAcl(quorum: String): Unit = { + @Test + def testDescribeShareGroupOffsetsWithoutTopicDescribeAcl(): Unit = { addAndVerifyAcls(shareGroupDescribeAcl(shareGroupResource), shareGroupResource) val request = describeShareGroupOffsetsRequest @@ -3451,9 +3343,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED, Errors.forCode(response.data.groups.get(0).topics.get(0).partitions.get(0).errorCode)) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithGroupDeleteAndTopicReadAcl(quorum: String): Unit = { + @Test + def testDeleteShareGroupOffsetsWithGroupDeleteAndTopicReadAcl(): Unit = { addAndVerifyAcls(shareGroupDeleteAcl(shareGroupResource), shareGroupResource) addAndVerifyAcls(topicReadAcl(topicResource), topicResource) @@ -3462,9 +3353,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithOperationAll(quorum: String): Unit = { + @Test + def testDeleteShareGroupOffsetsWithOperationAll(): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), shareGroupResource) addAndVerifyAcls(Set(allowAllOpsAcl), topicResource) @@ -3474,9 +3364,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithoutGroupDeleteOrTopicReadAcl(quorum: String): Unit = { + @Test + def testDeleteShareGroupOffsetsWithoutGroupDeleteOrTopicReadAcl(): Unit = { removeAllClientAcls() val request = deleteShareGroupOffsetsRequest @@ -3484,9 +3373,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithoutGroupDeleteAcl(quorum: String): Unit = { + @Test + def testDeleteShareGroupOffsetsWithoutGroupDeleteAcl(): Unit = { addAndVerifyAcls(topicReadAcl(topicResource), topicResource) val request = deleteShareGroupOffsetsRequest @@ -3494,9 +3382,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testDeleteShareGroupOffsetsWithoutTopicReadAcl(quorum: String): Unit = { + @Test + def testDeleteShareGroupOffsetsWithoutTopicReadAcl(): Unit = { addAndVerifyAcls(shareGroupDeleteAcl(shareGroupResource), shareGroupResource) val request = deleteShareGroupOffsetsRequest @@ -3568,9 +3455,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { removeAllClientAcls() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithGroupDescribeAndTopicDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupDescribeWithGroupDescribeAndTopicDescribeAcl(): Unit = { createConsumerGroupToDescribe() addAndVerifyAcls(groupDescribeAcl(groupResource), groupResource) @@ -3581,9 +3467,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithOperationAll(quorum: String): Unit = { + @Test + def testConsumerGroupDescribeWithOperationAll(): Unit = { createConsumerGroupToDescribe() val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) @@ -3595,9 +3480,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithoutGroupDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupDescribeWithoutGroupDescribeAcl(): Unit = { createConsumerGroupToDescribe() addAndVerifyAcls(topicDescribeAcl(topicResource), topicResource) @@ -3607,9 +3491,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithoutTopicDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupDescribeWithoutTopicDescribeAcl(): Unit = { createConsumerGroupToDescribe() addAndVerifyAcls(groupDescribeAcl(groupResource), groupResource) @@ -3619,9 +3502,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(quorum: String): Unit = { + @Test + def testConsumerGroupDescribeWithoutGroupDescribeOrTopicDescribeAcl(): Unit = { createConsumerGroupToDescribe() val request = consumerGroupDescribeRequest @@ -3757,9 +3639,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testPrefixAcls(quorum: String): Unit = { + @Test + def testPrefixAcls(): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)), new ResourcePattern(TOPIC, "f", PREFIXED)) addAndVerifyAcls(Set(new AccessControlEntry("User:otherPrincipal", WILDCARD_HOST, CREATE, DENY)), diff --git a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationWithOwnerTest.scala b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationWithOwnerTest.scala index b05162d49a0..34dc99ee48b 100644 --- a/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationWithOwnerTest.scala +++ b/core/src/test/scala/integration/kafka/api/DelegationTokenEndToEndAuthorizationWithOwnerTest.scala @@ -26,8 +26,7 @@ import org.apache.kafka.common.resource.ResourcePattern import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.security.token.delegation.DelegationToken import org.junit.jupiter.api.Assertions.{assertThrows, assertTrue} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.util.Collections import scala.concurrent.ExecutionException @@ -94,18 +93,16 @@ class DelegationTokenEndToEndAuthorizationWithOwnerTest extends DelegationTokenE createScramAdminClient(kafkaClientSaslMechanism, tokenRequesterPrincipal.getName, tokenRequesterPassword) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateTokenForOtherUserFails(quorum: String): Unit = { + @Test + def testCreateTokenForOtherUserFails(): Unit = { val thrown = assertThrows(classOf[ExecutionException], () => { createDelegationTokens(() => new CreateDelegationTokenOptions().owner(otherClientPrincipal), assert = false) }) assertTrue(thrown.getMessage.contains("Delegation Token authorization failed")) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeTokenForOtherUserFails(quorum: String): Unit = { + @Test + def testDescribeTokenForOtherUserFails(): Unit = { Using.resource(createScramAdminClient(kafkaClientSaslMechanism, describeTokenFailPrincipal.getName, describeTokenFailPassword)) { describeTokenFailAdminClient => Using.resource(createScramAdminClient(kafkaClientSaslMechanism, otherClientPrincipal.getName, otherClientPassword)) { otherClientAdminClient => otherClientAdminClient.createDelegationToken().delegationToken().get() @@ -117,9 +114,8 @@ class DelegationTokenEndToEndAuthorizationWithOwnerTest extends DelegationTokenE } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeTokenForOtherUserPasses(quorum: String): Unit = { + @Test + def testDescribeTokenForOtherUserPasses(): Unit = { val adminClient = createTokenRequesterAdminClient() try { val tokens = adminClient.describeDelegationToken( diff --git a/core/src/test/scala/integration/kafka/api/MetricsTest.scala b/core/src/test/scala/integration/kafka/api/MetricsTest.scala index e08801343fc..25702e04a0b 100644 --- a/core/src/test/scala/integration/kafka/api/MetricsTest.scala +++ b/core/src/test/scala/integration/kafka/api/MetricsTest.scala @@ -79,9 +79,9 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup { /** * Verifies some of the metrics of producer, consumer as well as server. */ - @ParameterizedTest(name = "testMetrics with systemRemoteStorageEnabled: {1}") - @CsvSource(Array("kraft, true", "kraft, false")) - def testMetrics(quorum: String, systemRemoteStorageEnabled: Boolean): Unit = { + @ParameterizedTest(name = "testMetrics with systemRemoteStorageEnabled: {0}") + @CsvSource(Array("true", "false")) + def testMetrics(systemRemoteStorageEnabled: Boolean): Unit = { val topic = "mytopic" createTopic(topic, numPartitions = 1, diff --git a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala index 0ab2328c53f..1fbda492d4f 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextAdminIntegrationTest.scala @@ -60,7 +60,7 @@ import org.apache.logging.log4j.core.config.Configurator import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{BeforeEach, Test, TestInfo, Timeout} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{MethodSource, ValueSource} +import org.junit.jupiter.params.provider.{MethodSource} import org.slf4j.LoggerFactory import java.util.AbstractMap.SimpleImmutableEntry @@ -2511,9 +2511,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testListGroups(unused: String): Unit = { + @Test + def testListGroups(): Unit = { val classicGroupId = "classic_group_id" val consumerGroupId = "consumer_group_id" val shareGroupId = "share_group_id" @@ -2643,9 +2642,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kip932")) - def testShareGroups(unused: String): Unit = { + def testShareGroups(): Unit = { val testGroupId = "test_group_id" val testClientId = "test_client_id" val fakeGroupId = "fake_group_id" diff --git a/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala index 1acd22dc3fa..18d34ad05e9 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextEndToEndAuthorizationTest.scala @@ -21,10 +21,8 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth._ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder import org.apache.kafka.clients.admin.AdminClientConfig -import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource import org.apache.kafka.common.errors.TopicAuthorizationException // This test case uses a separate listener for client and inter-broker communication, from @@ -88,9 +86,8 @@ class PlaintextEndToEndAuthorizationTest extends EndToEndAuthorizationTest { superuserClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers(interBrokerListenerName)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testListenerName(quorum: String): Unit = { + @Test + def testListenerName(): Unit = { // To check the client listener name, establish a session on the server by sending any request eg sendRecords val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords = 1, tp)) diff --git a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala index fe27443ca09..340c0b0cced 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerSendWhileDeletionTest.scala @@ -22,8 +22,7 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, Record import org.apache.kafka.common.TopicPartition import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.nio.charset.StandardCharsets import java.util @@ -50,9 +49,8 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * Producer will attempt to send messages to the partition specified in each record, and should * succeed as long as the partition is included in the metadata. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSendWithTopicDeletionMidWay(quorum: String): Unit = { + @Test + def testSendWithTopicDeletionMidWay(): Unit = { val numRecords = 10 val topic = "topic" @@ -91,9 +89,8 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * Producer will attempt to send messages to the partition specified in each record, and should * succeed as long as the metadata has been updated with new topic id. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSendWithRecreatedTopic(quorum: String): Unit = { + @Test + def testSendWithRecreatedTopic(): Unit = { val numRecords = 10 val topic = "topic" createTopic(topic) @@ -125,9 +122,8 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness { * Producer will attempt to send messages to the partition specified in each record, and should * succeed as long as the metadata cache on the leader includes the partition topic id. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSendWithTopicReassignmentIsMidWay(quorum: String): Unit = { + @Test + def testSendWithTopicReassignmentIsMidWay(): Unit = { val numRecords = 10 val topic = "topic" val partition0: TopicPartition = new TopicPartition(topic, 0) diff --git a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala index cf44bffbdc4..0c22def1bb2 100644 --- a/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslClientsWithInvalidCredentialsTest.scala @@ -23,7 +23,7 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.kafka.common.errors.SaslAuthenticationException -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.common.config.SaslConfigs @@ -91,7 +91,7 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { closeSasl() } - @ParameterizedTest(name="{displayName}.quorum=kraft.isIdempotenceEnabled={0}") + @ParameterizedTest(name="{displayName}.isIdempotenceEnabled={0}") @ValueSource(booleans = Array(true, false)) def testProducerWithAuthenticationFailure(isIdempotenceEnabled: Boolean): Unit = { val prop = new Properties() @@ -111,9 +111,8 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { verifyWithRetry(sendOneRecord(producer2))() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTransactionalProducerWithAuthenticationFailure(quorum: String): Unit = { + @Test + def testTransactionalProducerWithAuthenticationFailure(): Unit = { val txProducer = createTransactionalProducer() verifyAuthenticationException(txProducer.initTransactions()) @@ -157,9 +156,8 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest { verifyWithRetry(consumer.poll(Duration.ofMillis(1000)))(_.count == 1) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testKafkaAdminClientWithAuthenticationFailure(quorum: String): Unit = { + @Test + def testKafkaAdminClientWithAuthenticationFailure(): Unit = { val props = JaasTestUtils.adminClientSecurityConfigs(securityProtocol, OptionConverters.toJava(trustStoreFile), OptionConverters.toJava(clientSaslProperties)) props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()) val adminClient = Admin.create(props) diff --git a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala index b9136e9a75b..1ff40f4d266 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala @@ -34,9 +34,7 @@ import org.apache.kafka.common.{KafkaException, Uuid, requests} import org.apache.kafka.network.SocketServerConfigs import org.apache.kafka.server.config.QuotaConfig import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.io.IOException import java.net.{InetAddress, Socket} @@ -84,9 +82,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { } @Flaky("KAFKA-17999") - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDynamicConnectionQuota(quorum: String): Unit = { + @Test + def testDynamicConnectionQuota(): Unit = { val maxConnectionsPerIP = 5 def connectAndVerify(): Unit = { @@ -112,9 +109,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { verifyMaxConnections(maxConnectionsPerIPOverride, connectAndVerify) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDynamicListenerConnectionQuota(quorum: String): Unit = { + @Test + def testDynamicListenerConnectionQuota(): Unit = { val initialConnectionCount = connectionCount def connectAndVerify(): Unit = { @@ -185,9 +181,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDynamicListenerConnectionCreationRateQuota(quorum: String): Unit = { + @Test + def testDynamicListenerConnectionCreationRateQuota(): Unit = { // Create another listener. PLAINTEXT is an inter-broker listener // keep default limits val newListenerNames = Seq("PLAINTEXT", "EXTERNAL") @@ -247,9 +242,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest { waitForConnectionCount(initialConnectionCount) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDynamicIpConnectionRateQuota(quorum: String): Unit = { + @Test + def testDynamicIpConnectionRateQuota(): Unit = { val connRateLimit = 10 val initialConnectionCount = connectionCount // before setting connection rate to 10, verify we can do at least double that by default (no limit) diff --git a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala index 91bf8c03783..2273a69cf99 100644 --- a/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala +++ b/core/src/test/scala/integration/kafka/network/DynamicNumNetworkThreadsTest.scala @@ -24,9 +24,7 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.network.SocketServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.util.Properties import scala.jdk.CollectionConverters._ @@ -66,9 +64,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest { .count(listener == _.tags().get("listener")) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDynamicNumNetworkThreads(quorum: String): Unit = { + @Test + def testDynamicNumNetworkThreads(): Unit = { // Increase the base network thread count val newBaseNetworkThreadsCount = SocketServerConfigs.NUM_NETWORK_THREADS_DEFAULT + 1 var props = new Properties diff --git a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala index 68b8e1108ff..b17d81e018d 100644 --- a/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala +++ b/core/src/test/scala/integration/kafka/server/GssapiAuthenticationTest.scala @@ -38,9 +38,9 @@ import org.apache.kafka.common.security.kerberos.KerberosLogin import org.apache.kafka.common.utils.{LogContext, MockTime} import org.apache.kafka.network.SocketServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{MethodSource, ValueSource} +import org.junit.jupiter.params.provider.MethodSource import scala.jdk.CollectionConverters._ @@ -92,9 +92,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { * Tests that Kerberos replay error `Request is a replay (34)` is not handled as an authentication exception * since replay detection used to detect DoS attacks may occasionally reject valid concurrent requests. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testRequestIsAReplay(quorum: String): Unit = { + @Test + def testRequestIsAReplay(): Unit = { val successfulAuthsPerThread = 10 val futures = (0 until numThreads).map(_ => executor.submit(new Runnable { override def run(): Unit = verifyRetriableFailuresDuringAuthentication(successfulAuthsPerThread) @@ -110,9 +109,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { * are able to connect after the second re-login. Verifies that logout is performed only once * since duplicate logouts without successful login results in NPE from Java 9 onwards. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testLoginFailure(quorum: String): Unit = { + @Test + def testLoginFailure(): Unit = { val selector = createSelectorWithRelogin() try { val login = TestableKerberosLogin.instance @@ -134,9 +132,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { * is performed when credentials are unavailable between logout and login, we handle it as a * transient error and not an authentication failure so that clients may retry. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testReLogin(quorum: String): Unit = { + @Test + def testReLogin(): Unit = { val selector = createSelectorWithRelogin() try { val login = TestableKerberosLogin.instance @@ -166,9 +163,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup { * Tests that Kerberos error `Server not found in Kerberos database (7)` is handled * as a fatal authentication failure. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testServerNotFoundInKerberosDatabase(quorum: String): Unit = { + @Test + def testServerNotFoundInKerberosDatabase(): Unit = { val jaasConfig = clientConfig.getProperty(SaslConfigs.SASL_JAAS_CONFIG) val invalidServiceConfig = jaasConfig.replace("serviceName=\"kafka\"", "serviceName=\"invalid-service\"") clientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, invalidServiceConfig) diff --git a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala index 5e3f421caf2..d36c0ed1b6f 100755 --- a/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala +++ b/core/src/test/scala/integration/kafka/server/QuorumTestHarness.scala @@ -159,10 +159,6 @@ abstract class QuorumTestHarness extends Logging { private var testInfo: TestInfo = _ protected var implementation: QuorumImplementation = _ - def isShareGroupTest(): Boolean = { - TestInfoUtils.isShareGroupTest(testInfo) - } - def maybeGroupProtocolSpecified(): Option[GroupProtocol] = { TestInfoUtils.maybeGroupProtocolSpecified(testInfo) } diff --git a/core/src/test/scala/kafka/utils/TestInfoUtils.scala b/core/src/test/scala/kafka/utils/TestInfoUtils.scala index 83cbf869051..d82529e92b5 100644 --- a/core/src/test/scala/kafka/utils/TestInfoUtils.scala +++ b/core/src/test/scala/kafka/utils/TestInfoUtils.scala @@ -34,10 +34,6 @@ object TestInfoUtils { final val TestWithParameterizedGroupProtocolNames = "{displayName}.groupProtocol={0}" - def isShareGroupTest(testInfo: TestInfo): Boolean = { - testInfo.getDisplayName.contains("kip932") - } - def maybeGroupProtocolSpecified(testInfo: TestInfo): Option[GroupProtocol] = { if (testInfo.getDisplayName.contains("groupProtocol=classic")) Some(GroupProtocol.CLASSIC) diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index dc6595df87b..2259acd9c2b 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -25,9 +25,7 @@ import org.apache.kafka.clients.admin.{Admin, NewPartitions, NewTopic} import org.apache.kafka.common.errors.InvalidReplicaAssignmentException import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import java.util import java.util.Arrays.asList @@ -65,9 +63,8 @@ class AddPartitionsTest extends BaseRequestTest { admin = createAdminClient() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testWrongReplicaCount(quorum: String): Unit = { + @Test + def testWrongReplicaCount(): Unit = { assertEquals(classOf[InvalidReplicaAssignmentException], assertThrows(classOf[ExecutionException], () => { admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(2, singletonList(asList(0, 1, 2))))).all().get() @@ -78,9 +75,8 @@ class AddPartitionsTest extends BaseRequestTest { * Test that when we supply a manual partition assignment to createTopics, it must be 0-based * and consecutive. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMissingPartitionsInCreateTopics(quorum: String): Unit = { + @Test + def testMissingPartitionsInCreateTopics(): Unit = { val topic6Placements = new util.HashMap[Integer, util.List[Integer]] topic6Placements.put(1, asList(0, 1)) topic6Placements.put(2, asList(1, 0)) @@ -104,9 +100,8 @@ class AddPartitionsTest extends BaseRequestTest { * Test that when we supply a manual partition assignment to createPartitions, it must contain * enough partitions. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMissingPartitionsInCreatePartitions(quorum: String): Unit = { + @Test + def testMissingPartitionsInCreatePartitions(): Unit = { val cause = assertThrows(classOf[ExecutionException], () => admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(3, singletonList(asList(0, 1, 2))))).all().get()).getCause @@ -115,9 +110,8 @@ class AddPartitionsTest extends BaseRequestTest { "were specified."), "Unexpected error message: " + cause.getMessage) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementPartitions(quorum: String): Unit = { + @Test + def testIncrementPartitions(): Unit = { admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(3))).all().get() // wait until leader is elected @@ -144,9 +138,8 @@ class AddPartitionsTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testManualAssignmentOfReplicas(quorum: String): Unit = { + @Test + def testManualAssignmentOfReplicas(): Unit = { // Add 2 partitions admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3, asList(asList(0, 1), asList(2, 3))))).all().get() @@ -173,9 +166,8 @@ class AddPartitionsTest extends BaseRequestTest { assertEquals(Set(0, 1), replicas.asScala.toSet) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testReplicaPlacementAllServers(quorum: String): Unit = { + @Test + def testReplicaPlacementAllServers(): Unit = { admin.createPartitions(Collections.singletonMap(topic3, NewPartitions.increaseTo(7))).all().get() // read metadata from a broker and verify the new topic partitions exist @@ -201,9 +193,8 @@ class AddPartitionsTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testReplicaPlacementPartialServers(quorum: String): Unit = { + @Test + def testReplicaPlacementPartialServers(): Unit = { admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3))).all().get() // read metadata from a broker and verify the new topic partitions exist diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala index dd2cc239dfc..b0192b7e82f 100644 --- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala @@ -1732,9 +1732,8 @@ class PartitionTest extends AbstractPartitionTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = { + @Test + def testIsrNotExpandedIfReplicaIsFencedOrShutdown(): Unit = { val log = logManager.getOrCreateLog(topicPartition, topicId = topicId.toJava) seedLogData(log, numRecords = 10, leaderEpoch = 4) diff --git a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala index fb981369e6b..f730e35415f 100644 --- a/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala +++ b/core/src/test/scala/unit/kafka/integration/MinIsrConfigTest.scala @@ -23,17 +23,15 @@ import scala.collection.Seq import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.server.config.ServerLogConfigs -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test class MinIsrConfigTest extends KafkaServerTestHarness { val overridingProps = new Properties() overridingProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "5") def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1).map(KafkaConfig.fromProps(_, overridingProps)) - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDefaultKafkaConfig(quorum: String): Unit = { + @Test + def testDefaultKafkaConfig(): Unit = { assert(brokers.head.logManager.initialDefaultConfig.minInSyncReplicas == 5) } } diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index 0b838296811..d59f44e3e3c 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -36,9 +36,9 @@ import org.apache.kafka.common.utils.Time import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector} import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics -import org.junit.jupiter.api.Timeout +import org.junit.jupiter.api.{Test, Timeout} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{MethodSource, ValueSource} +import org.junit.jupiter.params.provider.MethodSource @Timeout(120) class MetricsTest extends KafkaServerTestHarness with Logging { @@ -56,9 +56,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging { val nMessages = 2 - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMetricsReporterAfterDeletingTopic(quorum: String): Unit = { + @Test + def testMetricsReporterAfterDeletingTopic(): Unit = { val topic = "test-topic-metric" createTopic(topic) deleteTopic(topic) @@ -66,9 +65,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testBrokerTopicMetricsUnregisteredAfterDeletingTopic(quorum: String): Unit = { + @Test + def testBrokerTopicMetricsUnregisteredAfterDeletingTopic(): Unit = { val topic = "test-broker-topic-metric" createTopic(topic, 2) // Produce a few messages to create the metrics @@ -81,33 +79,29 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testClusterIdMetric(quorum: String): Unit = { + @Test + def testClusterIdMetric(): Unit = { // Check if clusterId metric exists. val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=ClusterId"), 1) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testBrokerStateMetric(quorum: String): Unit = { + @Test + def testBrokerStateMetric(): Unit = { // Check if BrokerState metric exists. val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=BrokerState"), 1) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testYammerMetricsCountMetric(quorum: String): Unit = { + @Test + def testYammerMetricsCountMetric(): Unit = { // Check if yammer-metrics-count metric exists. val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=yammer-metrics-count"), 1) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testLinuxIoMetrics(quorum: String): Unit = { + @Test + def testLinuxIoMetrics(): Unit = { // Check if linux-disk-{read,write}-bytes metrics either do or do not exist depending on whether we are or are not // able to collect those metrics on the platform where this test is running. val usable = new LinuxIoMetricsCollector("/proc", Time.SYSTEM).usable() @@ -117,9 +111,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=$name"), expectedCount)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testJMXFilter(quorum: String): Unit = { + @Test + def testJMXFilter(): Unit = { // Check if cluster id metrics is not exposed in JMX assertTrue(ManagementFactory.getPlatformMBeanServer .isRegistered(new ObjectName("kafka.controller:type=KafkaController,name=ActiveControllerCount"))) @@ -127,9 +120,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging { .isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId"))) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUpdateJMXFilter(quorum: String): Unit = { + @Test + def testUpdateJMXFilter(): Unit = { // verify previously exposed metrics are removed and existing matching metrics are added brokers.foreach(broker => broker.kafkaYammerMetrics.reconfigure( Map(JmxReporter.EXCLUDE_CONFIG -> "kafka.controller:type=KafkaController,name=ActiveControllerCount").asJava @@ -140,9 +132,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging { .isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId"))) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testGeneralBrokerTopicMetricsAreGreedilyRegistered(quorum: String): Unit = { + @Test + def testGeneralBrokerTopicMetricsAreGreedilyRegistered(): Unit = { val topic = "test-broker-topic-metric" createTopic(topic, 2) @@ -156,9 +147,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertTrue(topicMetricGroups(topic).nonEmpty, "Topic metrics aren't registered") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testWindowsStyleTagNames(quorum: String): Unit = { + @Test + def testWindowsStyleTagNames(): Unit = { val path = "C:\\windows-path\\kafka-logs" val tags = Map("dir" -> path) val expectedMBeanName = Set(tags.keySet.head, ObjectName.quote(path)).mkString("=") @@ -213,9 +203,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertTrue(TestUtils.meterCount(bytesOut) > initialBytesOut) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testKRaftControllerMetrics(quorum: String): Unit = { + @Test + def testKRaftControllerMetrics(): Unit = { val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics Set( "kafka.controller:type=KafkaController,name=ActiveControllerCount", diff --git a/core/src/test/scala/unit/kafka/security/authorizer/AuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/authorizer/AuthorizerTest.scala index 0e2105f85d2..6a9d8dcd1d4 100644 --- a/core/src/test/scala/unit/kafka/security/authorizer/AuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/authorizer/AuthorizerTest.scala @@ -41,8 +41,6 @@ import org.apache.kafka.server.authorizer._ import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} import org.junit.jupiter.api.Test -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource import java.net.InetAddress import java.util @@ -52,7 +50,6 @@ import scala.jdk.CollectionConverters._ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { private final val PLAINTEXT = new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "127.0.0.1", 9020) - private final val KRAFT = "kraft" private val allowReadAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, READ, ALLOW) private val allowWriteAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, WRITE, ALLOW) @@ -104,32 +101,28 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { super.tearDown() } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAuthorizeThrowsOnNonLiteralResource(quorum: String): Unit = { + @Test + def testAuthorizeThrowsOnNonLiteralResource(): Unit = { assertThrows(classOf[IllegalArgumentException], () => authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "something", PREFIXED))) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAuthorizeWithEmptyResourceName(quorum: String): Unit = { + @Test + def testAuthorizeWithEmptyResourceName(): Unit = { assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(GROUP, "", LITERAL))) addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(GROUP, WILDCARD_RESOURCE, LITERAL)) assertTrue(authorize(authorizer1, requestContext, READ, new ResourcePattern(GROUP, "", LITERAL))) } // Authorizing the empty resource is not supported because empty resource name is invalid. - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testEmptyAclThrowsException(quorum: String): Unit = { + @Test + def testEmptyAclThrowsException(): Unit = { assertThrows(classOf[ApiException], () => addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(GROUP, "", LITERAL))) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testTopicAcl(quorum: String): Unit = { + @Test + def testTopicAcl(): Unit = { val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "rob") val user3 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "batman") @@ -183,9 +176,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { /** * CustomPrincipals should be compared with their principal type and name */ - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAllowAccessWithCustomPrincipal(quorum: String): Unit = { + @Test + def testAllowAccessWithCustomPrincipal(): Unit = { val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) val customUserPrincipal = new CustomPrincipal(KafkaPrincipal.USER_TYPE, username) val host1 = InetAddress.getByName("192.168.1.1") @@ -204,9 +196,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertFalse(authorize(authorizer1, host1Context, READ, resource), "User1 should not have READ access from host1 due to denyAcl") } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testDenyTakesPrecedence(quorum: String): Unit = { + @Test + def testDenyTakesPrecedence(): Unit = { val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) val host = InetAddress.getByName("192.168.2.1") val session = newRequestContext(user, host) @@ -220,9 +211,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertFalse(authorize(authorizer1, session, READ, resource), "deny should take precedence over allow.") } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAllowAllAccess(quorum: String): Unit = { + @Test + def testAllowAllAccess(): Unit = { val allowAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, ALLOW) changeAclAndVerify(Set.empty, Set(allowAllAcl), Set.empty) @@ -231,9 +221,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertTrue(authorize(authorizer1, context, READ, resource), "allow all acl should allow access to all.") } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testSuperUserHasAccess(quorum: String): Unit = { + @Test + def testSuperUserHasAccess(): Unit = { val denyAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, DENY) changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty) @@ -248,9 +237,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { /** * CustomPrincipals should be compared with their principal type and name */ - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testSuperUserWithCustomPrincipalHasAccess(quorum: String): Unit = { + @Test + def testSuperUserWithCustomPrincipalHasAccess(): Unit = { val denyAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, DENY) changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty) @@ -259,9 +247,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertTrue(authorize(authorizer1, session, READ, resource), "superuser with custom principal always has access, no matter what acls.") } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testWildCardAcls(quorum: String): Unit = { + @Test + def testWildCardAcls(): Unit = { assertFalse(authorize(authorizer1, requestContext, READ, resource), "when acls = [], authorizer should fail close.") val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) @@ -284,15 +271,13 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertFalse(authorize(authorizer1, host1Context, WRITE, resource), "User1 should not have WRITE access from host1") } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testNoAclFound(quorum: String): Unit = { + @Test + def testNoAclFound(): Unit = { assertFalse(authorize(authorizer1, requestContext, READ, resource), "when acls = [], authorizer should deny op.") } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testNoAclFoundOverride(quorum: String): Unit = { + @Test + def testNoAclFoundOverride(): Unit = { val props = properties props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true") @@ -307,9 +292,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { } } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAclConfigWithWhitespace(quorum: String): Unit = { + @Test + def testAclConfigWithWhitespace(): Unit = { val props = properties props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, " true") // replace all property values with leading & trailing whitespaces @@ -325,9 +309,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { } } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAclManagementAPIs(quorum: String): Unit = { + @Test + def testAclManagementAPIs(): Unit = { val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob") val host1 = "host1" @@ -393,9 +376,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { /** * Test ACL inheritance, as described in #{org.apache.kafka.common.acl.AclOperation} */ - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAclInheritance(quorum: String): Unit = { + @Test + def testAclInheritance(): Unit = { testImplicationsOfAllow(AclOperation.ALL, Set(READ, WRITE, CREATE, DELETE, ALTER, DESCRIBE, CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, CREATE_TOKENS, DESCRIBE_TOKENS, TWO_PHASE_COMMIT)) testImplicationsOfDeny(AclOperation.ALL, Set(READ, WRITE, CREATE, DELETE, ALTER, DESCRIBE, @@ -442,17 +424,15 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { removeAcls(authorizer1, acls, clusterResource) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAccessAllowedIfAllowAclExistsOnWildcardResource(quorum: String): Unit = { + @Test + def testAccessAllowedIfAllowAclExistsOnWildcardResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl), wildCardResource) assertTrue(authorize(authorizer1, requestContext, READ, resource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testDeleteAclOnWildcardResource(quorum: String): Unit = { + @Test + def testDeleteAclOnWildcardResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), wildCardResource) removeAcls(authorizer1, Set(allowReadAcl), wildCardResource) @@ -460,9 +440,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set(allowWriteAcl), getAcls(authorizer1, wildCardResource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testDeleteAllAclOnWildcardResource(quorum: String): Unit = { + @Test + def testDeleteAllAclOnWildcardResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl), wildCardResource) removeAcls(authorizer1, Set.empty, wildCardResource) @@ -470,17 +449,15 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set.empty, getAcls(authorizer1)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAccessAllowedIfAllowAclExistsOnPrefixedResource(quorum: String): Unit = { + @Test + def testAccessAllowedIfAllowAclExistsOnPrefixedResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl), prefixedResource) assertTrue(authorize(authorizer1, requestContext, READ, resource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testDeleteAclOnPrefixedResource(quorum: String): Unit = { + @Test + def testDeleteAclOnPrefixedResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource) removeAcls(authorizer1, Set(allowReadAcl), prefixedResource) @@ -488,9 +465,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set(allowWriteAcl), getAcls(authorizer1, prefixedResource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testDeleteAllAclOnPrefixedResource(quorum: String): Unit = { + @Test + def testDeleteAllAclOnPrefixedResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource) removeAcls(authorizer1, Set.empty, prefixedResource) @@ -498,9 +474,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set.empty, getAcls(authorizer1)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAddAclsOnLiteralResource(quorum: String): Unit = { + @Test + def testAddAclsOnLiteralResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), resource) addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), resource) @@ -509,9 +484,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set.empty, getAcls(authorizer1, prefixedResource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAddAclsOnWildcardResource(quorum: String): Unit = { + @Test + def testAddAclsOnWildcardResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), wildCardResource) addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), wildCardResource) @@ -520,9 +494,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set.empty, getAcls(authorizer1, prefixedResource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAddAclsOnPrefixedResource(quorum: String): Unit = { + @Test + def testAddAclsOnPrefixedResource(): Unit = { addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource) addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), prefixedResource) @@ -531,9 +504,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set.empty, getAcls(authorizer1, resource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAuthorizeWithPrefixedResource(quorum: String): Unit = { + @Test + def testAuthorizeWithPrefixedResource(): Unit = { addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "a_other", LITERAL)) addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "a_other", PREFIXED)) addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), PREFIXED)) @@ -552,9 +524,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertTrue(authorize(authorizer1, requestContext, READ, resource)) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testSingleCharacterResourceAcls(quorum: String): Unit = { + @Test + def testSingleCharacterResourceAcls(): Unit = { addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(TOPIC, "f", LITERAL)) assertTrue(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "f", LITERAL))) assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "foo", LITERAL))) @@ -565,9 +536,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "foo_", LITERAL))) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testGetAclsPrincipal(quorum: String): Unit = { + @Test + def testGetAclsPrincipal(): Unit = { val aclOnSpecificPrincipal = new AccessControlEntry(principal.toString, WILDCARD_HOST, WRITE, ALLOW) addAcls(authorizer1, Set(aclOnSpecificPrincipal), resource) @@ -586,9 +556,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(0, getAcls(authorizer1, principal).size, "acl on wildcard should not be returned for specific request") } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAclsFilter(quorum: String): Unit = { + @Test + def testAclsFilter(): Unit = { val resource1 = new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), LITERAL) val resource2 = new ResourcePattern(TOPIC, "bar-" + UUID.randomUUID(), LITERAL) val prefixedResource = new ResourcePattern(TOPIC, "bar-", PREFIXED) @@ -622,9 +591,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest { assertEquals(Set.empty, deleteResults(3).aclBindingDeleteResults.asScala.map(_.aclBinding).toSet) } - @ParameterizedTest - @ValueSource(strings = Array(KRAFT)) - def testAuthorizeByResourceTypeNoAclFoundOverride(quorum: String): Unit = { + @Test + def testAuthorizeByResourceTypeNoAclFoundOverride(): Unit = { val props = properties props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true") diff --git a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala index 1b35f939619..406609239a0 100644 --- a/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AddPartitionsToTxnRequestServerTest.scala @@ -32,9 +32,9 @@ import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, FindCoordinatorRequest, FindCoordinatorResponse, InitProducerIdRequest, InitProducerIdResponse} import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{Arguments, MethodSource, ValueSource} +import org.junit.jupiter.params.provider.{Arguments, MethodSource} import scala.collection.mutable import scala.jdk.CollectionConverters._ @@ -55,7 +55,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { @ParameterizedTest @MethodSource(value = Array("parameters")) - def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(quorum: String, version: Short): Unit = { + def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(version: Short): Unit = { // The basic idea is that we have one unknown topic and one created topic. We should get the 'UNKNOWN_TOPIC_OR_PARTITION' // error for the unknown topic and the 'OPERATION_NOT_ATTEMPTED' error for the known and authorized topic. val nonExistentTopic = new TopicPartition("unknownTopic", 0) @@ -110,9 +110,8 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testOneSuccessOneErrorInBatchedRequest(quorum: String): Unit = { + @Test + def testOneSuccessOneErrorInBatchedRequest(): Unit = { val tp0 = new TopicPartition(topic1, 0) val transactionalId1 = "foobar" val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator @@ -149,9 +148,8 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest { assertEquals(expectedErrors, errors) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testVerifyOnly(quorum: String): Unit = { + @Test + def testVerifyOnly(): Unit = { val tp0 = new TopicPartition(topic1, 0) val transactionalId = "foobar" @@ -209,7 +207,7 @@ object AddPartitionsToTxnRequestServerTest { def parameters: JStream[Arguments] = { val arguments = mutable.ListBuffer[Arguments]() ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions().forEach { version => - arguments += Arguments.of("kraft", version) + arguments += Arguments.of(version) } arguments.asJava.stream() } diff --git a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala index 8e2698b0842..f5bd92ce15e 100644 --- a/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterReplicaLogDirsRequestTest.scala @@ -27,8 +27,7 @@ import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplic import org.apache.kafka.server.config.ServerLogConfigs import org.apache.kafka.storage.internals.log.LogFileUtils import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.util.Properties import scala.jdk.CollectionConverters._ @@ -52,9 +51,8 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { .find(p => p.partitionIndex == tp.partition).get.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterReplicaLogDirsRequest(quorum: String): Unit = { + @Test + def testAlterReplicaLogDirsRequest(): Unit = { val partitionNum = 5 // Alter replica dir before topic creation @@ -88,9 +86,8 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterReplicaLogDirsRequestErrorCode(quorum: String): Unit = { + @Test + def testAlterReplicaLogDirsRequestErrorCode(): Unit = { val offlineDir = new File(brokers.head.config.logDirs.tail.head).getAbsolutePath val validDir1 = new File(brokers.head.config.logDirs(1)).getAbsolutePath val validDir2 = new File(brokers.head.config.logDirs(2)).getAbsolutePath @@ -127,9 +124,8 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest { assertEquals(Errors.KAFKA_STORAGE_ERROR, findErrorForPartition(alterReplicaDirResponse3, new TopicPartition(topic, 2))) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterReplicaLogDirsRequestWithRetention(quorum: String): Unit = { + @Test + def testAlterReplicaLogDirsRequestWithRetention(): Unit = { val partitionNum = 1 // Alter replica dir before topic creation diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala index 7ea3052925b..99231470b12 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestNotAuthorizedTest.scala @@ -25,9 +25,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse} import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.TestInfo -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{Test, TestInfo} import java.util import java.util.Properties @@ -54,9 +52,8 @@ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest private val user1 = "user1" private val user2 = "user2" - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterNothingNotAuthorized(quorum: String): Unit = { + @Test + def testAlterNothingNotAuthorized(): Unit = { val request = new AlterUserScramCredentialsRequest.Builder( new AlterUserScramCredentialsRequestData() .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion]) @@ -67,9 +64,8 @@ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest assertEquals(0, results.size) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterSomethingNotAuthorized(quorum: String): Unit = { + @Test + def testAlterSomethingNotAuthorized(): Unit = { val request = new AlterUserScramCredentialsRequest.Builder( new AlterUserScramCredentialsRequestData() .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`))) diff --git a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala index 4ebe65ec9a9..16c6203bac5 100644 --- a/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AlterUserScramCredentialsRequestTest.scala @@ -34,7 +34,7 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult} import org.apache.kafka.server.common.MetadataVersion import org.apache.kafka.server.config.ServerConfigs -import org.junit.jupiter.api.{BeforeEach, TestInfo} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.ValueSource @@ -79,9 +79,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { private val user3 = "user3@user3.com" private val unknownUser = "unknownUser" - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterNothing(quorum: String): Unit = { + @Test + def testAlterNothing(): Unit = { val request = new AlterUserScramCredentialsRequest.Builder( new AlterUserScramCredentialsRequestData() .setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion]) @@ -92,9 +91,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { assertEquals(0, results.size) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterSameThingTwice(quorum: String): Unit = { + @Test + def testAlterSameThingTwice(): Unit = { val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`) val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`) val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`) @@ -133,9 +131,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { }) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterEmptyUser(quorum: String): Unit = { + @Test + def testAlterEmptyUser(): Unit = { val deletionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`) val upsertionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`) .setIterations(4096).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes) @@ -162,9 +159,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { }) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterUnknownMechanism(quorum: String): Unit = { + @Test + def testAlterUnknownMechanism(): Unit = { val deletionUnknown1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.UNKNOWN.`type`) val deletionValid1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`) val deletionUnknown2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(10.toByte) @@ -190,9 +186,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { results.asScala.foreach(result => assertEquals("Unknown SCRAM mechanism", result.errorMessage)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterTooFewIterations(quorum: String): Unit = { + @Test + def testAlterTooFewIterations(): Unit = { val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1) .setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(1) .setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes) @@ -207,9 +202,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { assertEquals("Too few iterations", results.get(0).errorMessage) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterTooManyIterations(quorum: String): Unit = { + @Test + def testAlterTooManyIterations(): Unit = { val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1) .setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(Integer.MAX_VALUE) .setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes) @@ -224,9 +218,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { assertEquals("Too many iterations", results.get(0).errorMessage) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteSomethingThatDoesNotExist(quorum: String): Unit = { + @Test + def testDeleteSomethingThatDoesNotExist(): Unit = { val request = new AlterUserScramCredentialsRequest.Builder( new AlterUserScramCredentialsRequestData() .setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`))) @@ -238,9 +231,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest { checkAllErrorsAlteringCredentials(results, Errors.RESOURCE_NOT_FOUND, "when deleting a non-existing credential") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAlterAndDescribe(quorum: String): Unit = { + @Test + def testAlterAndDescribe(): Unit = { // create a bunch of credentials val request1_0 = new AlterUserScramCredentialsRequest.Builder( new AlterUserScramCredentialsRequestData() diff --git a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala index f63434a2561..caaf49ba569 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerMutationQuotaTest.scala @@ -49,9 +49,7 @@ import org.apache.kafka.test.{TestUtils => JTestUtils} import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Assertions.assertTrue import org.junit.jupiter.api.Assertions.fail -import org.junit.jupiter.api.{BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import scala.collection.Seq import scala.jdk.CollectionConverters._ @@ -126,9 +124,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { waitUserQuota(ThrottledPrincipal.getName, ControllerMutationRate) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSetUnsetQuota(quorum: String): Unit = { + @Test + def testSetUnsetQuota(): Unit = { val rate = 1.5 val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "User") // Default Value @@ -143,9 +140,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { waitUserQuota(principal.getName, Long.MaxValue) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testQuotaMetric(quorum: String): Unit = { + @Test + def testQuotaMetric(): Unit = { asPrincipal(ThrottledPrincipal) { // Metric is lazily created assertTrue(quotaMetric(principal.getName).isEmpty) @@ -166,9 +162,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testStrictCreateTopicsRequest(quorum: String): Unit = { + @Test + def testStrictCreateTopicsRequest(): Unit = { asPrincipal(ThrottledPrincipal) { // Create two topics worth of 30 partitions each. As we use a strict quota, we // expect one to be created and one to be rejected. @@ -190,9 +185,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testPermissiveCreateTopicsRequest(quorum: String): Unit = { + @Test + def testPermissiveCreateTopicsRequest(): Unit = { asPrincipal(ThrottledPrincipal) { // Create two topics worth of 30 partitions each. As we use a permissive quota, we // expect both topics to be created. @@ -204,9 +198,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnboundedCreateTopicsRequest(quorum: String): Unit = { + @Test + def testUnboundedCreateTopicsRequest(): Unit = { asPrincipal(UnboundedPrincipal) { // Create two topics worth of 30 partitions each. As we use an user without quota, we // expect both topics to be created. The throttle time should be equal to 0. @@ -216,9 +209,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testStrictDeleteTopicsRequest(quorum: String): Unit = { + @Test + def testStrictDeleteTopicsRequest(): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion) } @@ -244,9 +236,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testPermissiveDeleteTopicsRequest(quorum: String): Unit = { + @Test + def testPermissiveDeleteTopicsRequest(): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion) } @@ -262,9 +253,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnboundedDeleteTopicsRequest(quorum: String): Unit = { + @Test + def testUnboundedDeleteTopicsRequest(): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion) @@ -276,9 +266,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testStrictCreatePartitionsRequest(quorum: String): Unit = { + @Test + def testStrictCreatePartitionsRequest(): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion) } @@ -304,9 +293,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testPermissiveCreatePartitionsRequest(quorum: String): Unit = { + @Test + def testPermissiveCreatePartitionsRequest(): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion) } @@ -322,9 +310,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnboundedCreatePartitionsRequest(quorum: String): Unit = { + @Test + def testUnboundedCreatePartitionsRequest(): Unit = { asPrincipal(UnboundedPrincipal) { createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion) diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index ca61193425a..df2ef30ee45 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -24,16 +24,14 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCol import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.CreateTopicsRequest import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import scala.jdk.CollectionConverters._ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testValidCreateTopicsRequests(quorum: String): Unit = { + @Test + def testValidCreateTopicsRequests(): Unit = { // Generated assignments validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1")))) validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic2", replicationFactor = 3)))) @@ -61,9 +59,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { topicReq("topic14", replicationFactor = -1, numPartitions = 2)))) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testErrorCreateTopicsRequests(quorum: String): Unit = { + @Test + def testErrorCreateTopicsRequests(): Unit = { val existingTopic = "existing-topic" createTopic(existingTopic) // Basic @@ -99,9 +96,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { validateTopicExists("partial-none") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testInvalidCreateTopicsRequests(quorum: String): Unit = { + @Test + def testInvalidCreateTopicsRequests(): Unit = { // Partitions/ReplicationFactor and ReplicaAssignment validateErrorCreateTopicsRequests(topicsReq(Seq( topicReq("bad-args-topic", numPartitions = 10, replicationFactor = 3, @@ -114,9 +110,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { Map("bad-args-topic" -> error(Errors.INVALID_REQUEST)), checkErrorMessage = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateTopicsRequestVersions(quorum: String): Unit = { + @Test + def testCreateTopicsRequestVersions(): Unit = { for (version <- ApiKeys.CREATE_TOPICS.oldestVersion to ApiKeys.CREATE_TOPICS.latestVersion) { val topic = s"topic_$version" val data = new CreateTopicsRequestData() @@ -153,9 +148,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateClusterMetadataTopic(quorum: String): Unit = { + @Test + def testCreateClusterMetadataTopic(): Unit = { validateErrorCreateTopicsRequests( topicsReq(Seq(topicReq(Topic.CLUSTER_METADATA_TOPIC_NAME))), Map(Topic.CLUSTER_METADATA_TOPIC_NAME -> diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala index 96ebfd66683..02fde8135f8 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala @@ -26,9 +26,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG import org.apache.kafka.server.policy.CreateTopicPolicy import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata -import org.junit.jupiter.api.TestInfo -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{Test, TestInfo} import scala.jdk.CollectionConverters._ @@ -46,9 +44,8 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest Seq(properties) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testValidCreateTopicsRequests(quorum: String): Unit = { + @Test + def testValidCreateTopicsRequests(): Unit = { validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1", numPartitions = 5)))) @@ -65,9 +62,8 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest assignment = Map(0 -> List(1, 0), 1 -> List(0, 1)))))) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testErrorCreateTopicsRequests(quorum: String): Unit = { + @Test + def testErrorCreateTopicsRequests(): Unit = { val existingTopic = "existing-topic" createTopic(existingTopic, 5) diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala index b4c8d922dd9..8e20f98aaaf 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsOnPlainTextTest.scala @@ -21,10 +21,8 @@ import kafka.security.JaasTestUtils import java.util import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.errors.UnsupportedByAuthenticationException -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions.assertThrows -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource import scala.concurrent.ExecutionException import scala.jdk.javaapi.OptionConverters @@ -48,9 +46,8 @@ class DelegationTokenRequestsOnPlainTextTest extends BaseRequestTest { config } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDelegationTokenRequests(quorum: String): Unit = { + @Test + def testDelegationTokenRequests(): Unit = { adminClient = Admin.create(createAdminConfig) val createResult = adminClient.createDelegationToken() diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala index 2c211eb042a..38040d0a120 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsTest.scala @@ -25,9 +25,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.common.utils.SecurityUtils import org.apache.kafka.server.config.DelegationTokenManagerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.util import scala.concurrent.ExecutionException @@ -65,9 +63,8 @@ class DelegationTokenRequestsTest extends IntegrationTestHarness with SaslSetup config } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDelegationTokenRequests(quorum: String): Unit = { + @Test + def testDelegationTokenRequests(): Unit = { adminClient = Admin.create(createAdminConfig) // create token1 with renewer1 diff --git a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala index c380816f769..bb55cf33ffd 100644 --- a/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala +++ b/core/src/test/scala/unit/kafka/server/DelegationTokenRequestsWithDisableTokenFeatureTest.scala @@ -22,9 +22,7 @@ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig} import org.apache.kafka.common.errors.DelegationTokenDisabledException import org.apache.kafka.common.security.auth.SecurityProtocol import org.junit.jupiter.api.Assertions.assertThrows -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.util import scala.concurrent.ExecutionException @@ -55,9 +53,8 @@ class DelegationTokenRequestsWithDisableTokenFeatureTest extends BaseRequestTest config } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDelegationTokenRequests(quorum: String): Unit = { + @Test + def testDelegationTokenRequests(): Unit = { adminClient = Admin.create(createAdminConfig) val createResult = adminClient.createDelegationToken() diff --git a/core/src/test/scala/unit/kafka/server/DeleteRecordsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteRecordsRequestTest.scala index 587fc4e5e62..de2eb967d97 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteRecordsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteRecordsRequestTest.scala @@ -25,8 +25,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DeleteRecordsRequest, DeleteRecordsResponse} import org.apache.kafka.common.serialization.StringSerializer import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.util.Collections import java.util.concurrent.TimeUnit @@ -36,9 +35,8 @@ class DeleteRecordsRequestTest extends BaseRequestTest { private val TIMEOUT_MS = 1000 private val MESSAGES_PRODUCED_PER_PARTITION = 10 - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteRecordsHappyCase(quorum: String): Unit = { + @Test + def testDeleteRecordsHappyCase(): Unit = { val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords // Create the DeleteRecord request requesting deletion of offset which is not present @@ -61,9 +59,8 @@ class DeleteRecordsRequestTest extends BaseRequestTest { validateLogStartOffsetForTopic(topicPartition, offsetToDelete) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testErrorWhenDeletingRecordsWithInvalidOffset(quorum: String): Unit = { + @Test + def testErrorWhenDeletingRecordsWithInvalidOffset(): Unit = { val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords // Create the DeleteRecord request requesting deletion of offset which is not present @@ -86,9 +83,8 @@ class DeleteRecordsRequestTest extends BaseRequestTest { validateLogStartOffsetForTopic(topicPartition, 0) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testErrorWhenDeletingRecordsWithInvalidTopic(quorum: String): Unit = { + @Test + def testErrorWhenDeletingRecordsWithInvalidTopic(): Unit = { val invalidTopicPartition = new TopicPartition("invalid-topic", 0) // Create the DeleteRecord request requesting deletion of offset which is not present val offsetToDelete = 1 diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala index ca088dd1680..d51aba04c8d 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestTest.scala @@ -28,17 +28,15 @@ import org.apache.kafka.common.requests.DeleteTopicsResponse import org.apache.kafka.common.requests.MetadataRequest import org.apache.kafka.common.requests.MetadataResponse import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import scala.collection.Seq import scala.jdk.CollectionConverters._ class DeleteTopicsRequestTest extends BaseRequestTest with Logging { - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTopicDeletionClusterHasOfflinePartitions(quorum: String): Unit = { + @Test + def testTopicDeletionClusterHasOfflinePartitions(): Unit = { // Create two topics with one partition/replica. Make one of them offline. val offlineTopic = "topic-1" val onlineTopic = "topic-2" @@ -70,9 +68,8 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging { "The topics are found in the Broker's cache") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testValidDeleteTopicRequests(quorum: String): Unit = { + @Test + def testValidDeleteTopicRequests(): Unit = { val timeout = 10000 // Single topic createTopic("topic-1") @@ -138,9 +135,8 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging { connectAndReceive[DeleteTopicsResponse](request, destination = socketServer) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteTopicsVersions(quorum: String): Unit = { + @Test + def testDeleteTopicsVersions(): Unit = { val timeout = 10000 for (version <- ApiKeys.DELETE_TOPICS.oldestVersion to ApiKeys.DELETE_TOPICS.latestVersion) { info(s"Creating and deleting tests for version $version") diff --git a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala index 4232030634c..d8d654082e9 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteTopicsRequestWithDeletionDisabledTest.scala @@ -25,9 +25,7 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse} import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.TestInfo -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{Test, TestInfo} class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest { @@ -48,9 +46,8 @@ class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest { props.map(KafkaConfig.fromProps) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDeleteRecordsRequest(quorum: String): Unit = { + @Test + def testDeleteRecordsRequest(): Unit = { val topic = "topic-1" val request = new DeleteTopicsRequest.Builder( new DeleteTopicsRequestData() diff --git a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala index 6e43f904c11..1d3048cec6a 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeClusterRequestTest.scala @@ -27,9 +27,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.{BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import java.lang.{Byte => JByte} import java.util.Properties @@ -48,15 +46,13 @@ class DescribeClusterRequestTest extends BaseRequestTest { doSetup(testInfo, createOffsetsTopic = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeClusterRequestIncludingClusterAuthorizedOperations(quorum: String): Unit = { + @Test + def testDescribeClusterRequestIncludingClusterAuthorizedOperations(): Unit = { testDescribeClusterRequest(true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeClusterRequestExcludingClusterAuthorizedOperations(quorum: String): Unit = { + @Test + def testDescribeClusterRequestExcludingClusterAuthorizedOperations(): Unit = { testDescribeClusterRequest(false) } diff --git a/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala index d22b53c0cb6..c7efd608dfd 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeLogDirsRequestTest.scala @@ -25,8 +25,7 @@ import org.apache.kafka.common.message.DescribeLogDirsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import scala.jdk.CollectionConverters._ @@ -39,9 +38,8 @@ class DescribeLogDirsRequestTest extends BaseRequestTest { val tp0 = new TopicPartition(topic, 0) val tp1 = new TopicPartition(topic, 1) - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeLogDirsRequest(quorum: String): Unit = { + @Test + def testDescribeLogDirsRequest(): Unit = { val onlineDir = new File(brokers.head.config.logDirs.head).getAbsolutePath val offlineDir = new File(brokers.head.config.logDirs.tail.head).getAbsolutePath brokers.head.replicaManager.handleLogDirFailure(offlineDir) diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala index 7a4a885d9ed..418753e4a6e 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestNotAuthorizedTest.scala @@ -24,8 +24,7 @@ import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, De import org.apache.kafka.metadata.authorizer.StandardAuthorizer import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.util.Properties @@ -39,9 +38,8 @@ class DescribeUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTe properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeNotAuthorized(quorum: String): Unit = { + @Test + def testDescribeNotAuthorized(): Unit = { val request = new DescribeUserScramCredentialsRequest.Builder( new DescribeUserScramCredentialsRequestData()).build() val response = sendDescribeUserScramCredentialsRequest(request) diff --git a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala index bc8bd5a13ac..850cca028e5 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeUserScramCredentialsRequestTest.scala @@ -28,8 +28,6 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource import scala.jdk.CollectionConverters._ @@ -48,9 +46,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest { super.setUp(testInfo) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeNothing(quorum: String): Unit = { + @Test + def testDescribeNothing(): Unit = { val request = new DescribeUserScramCredentialsRequest.Builder( new DescribeUserScramCredentialsRequestData()).build() val response = sendDescribeUserScramCredentialsRequest(request) @@ -60,9 +57,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest { assertEquals(0, response.data.results.size, "Expected no credentials when describing everything and there are no credentials") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeWithNull(quorum: String): Unit = { + @Test + def testDescribeWithNull(): Unit = { val request = new DescribeUserScramCredentialsRequest.Builder( new DescribeUserScramCredentialsRequestData().setUsers(null)).build() val response = sendDescribeUserScramCredentialsRequest(request) @@ -82,9 +78,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest { assertEquals(Errors.NONE.code, error, "Did not expect controller error when routed to non-controller") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeSameUserTwice(quorum: String): Unit = { + @Test + def testDescribeSameUserTwice(): Unit = { val user = "user1" val userName = new UserName().setName(user) val request = new DescribeUserScramCredentialsRequest.Builder( @@ -98,9 +93,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest { assertEquals(s"Cannot describe SCRAM credentials for the same user twice in a single request: $user", result.errorMessage) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnknownUser(quorum: String): Unit = { + @Test + def testUnknownUser(): Unit = { val unknownUser = "unknownUser" val request = new DescribeUserScramCredentialsRequest.Builder( new DescribeUserScramCredentialsRequestData().setUsers(List(new UserName().setName(unknownUser)).asJava)).build() diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index 28c8d694f9d..61105b176dc 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -40,8 +40,6 @@ import org.apache.kafka.storage.internals.log.{LogConfig, UnifiedLog} import org.apache.kafka.test.TestUtils.assertFutureThrows import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{Test, Timeout} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource import org.mockito.ArgumentCaptor import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ @@ -61,9 +59,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { List(KafkaConfig.fromProps(cfg)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConfigChange(quorum: String): Unit = { + @Test + def testConfigChange(): Unit = { val oldVal: java.lang.Long = 100000L val newVal: java.lang.Long = 200000L val tp = new TopicPartition("test", 0) @@ -95,9 +92,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDynamicTopicConfigChange(quorum: String): Unit = { + @Test + def testDynamicTopicConfigChange(): Unit = { val tp = new TopicPartition("test", 0) val oldSegmentSize = 1000 val logProps = new Properties() @@ -180,59 +176,52 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testClientIdQuotaConfigChange(quorum: String): Unit = { + @Test + def testClientIdQuotaConfigChange(): Unit = { val m = new util.HashMap[String, String] m.put(CLIENT_ID, "testClient") testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUserQuotaConfigChange(quorum: String): Unit = { + @Test + def testUserQuotaConfigChange(): Unit = { val m = new util.HashMap[String, String] m.put(USER, "ANONYMOUS") testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUserClientIdQuotaChange(quorum: String): Unit = { + @Test + def testUserClientIdQuotaChange(): Unit = { val m = new util.HashMap[String, String] m.put(USER, "ANONYMOUS") m.put(CLIENT_ID, "testClient") testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDefaultClientIdQuotaConfigChange(quorum: String): Unit = { + @Test + def testDefaultClientIdQuotaConfigChange(): Unit = { val m = new util.HashMap[String, String] m.put(CLIENT_ID, null) testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDefaultUserQuotaConfigChange(quorum: String): Unit = { + @Test + def testDefaultUserQuotaConfigChange(): Unit = { val m = new util.HashMap[String, String] m.put(USER, null) testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDefaultUserClientIdQuotaConfigChange(quorum: String): Unit = { + @Test + def testDefaultUserClientIdQuotaConfigChange(): Unit = { val m = new util.HashMap[String, String] m.put(USER, null) m.put(CLIENT_ID, null) testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIpQuotaInitialization(quorum: String): Unit = { + @Test + def testIpQuotaInitialization(): Unit = { val broker = brokers.head val admin = createAdminClient() try { @@ -252,9 +241,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIpQuotaConfigChange(quorum: String): Unit = { + @Test + def testIpQuotaConfigChange(): Unit = { val admin = createAdminClient() try { val alterations = util.Arrays.asList( @@ -296,9 +284,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { private def tempTopic() : String = "testTopic" + random.nextInt(1000000) - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConfigChangeOnNonExistingTopicWithAdminClient(quorum: String): Unit = { + @Test + def testConfigChangeOnNonExistingTopicWithAdminClient(): Unit = { val topic = tempTopic() val admin = createAdminClient() try { @@ -314,9 +301,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterDefaultTopicConfig(quorum: String): Unit = { + @Test + def testIncrementalAlterDefaultTopicConfig(): Unit = { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.TOPIC, "") @@ -346,9 +332,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testBrokerIdConfigChangeAndDelete(quorum: String): Unit = { + @Test + def testBrokerIdConfigChangeAndDelete(): Unit = { val newValue: Long = 100000L val brokerId: String = this.brokers.head.config.brokerId.toString setBrokerConfigs(brokerId, newValue) @@ -370,9 +355,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDefaultBrokerIdConfigChangeAndDelete(quorum: String): Unit = { + @Test + def testDefaultBrokerIdConfigChangeAndDelete(): Unit = { val newValue: Long = 100000L val brokerId: String = "" setBrokerConfigs(brokerId, newValue) @@ -393,9 +377,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDefaultAndBrokerIdConfigChange(quorum: String): Unit = { + @Test + def testDefaultAndBrokerIdConfigChange(): Unit = { val newValue: Long = 100000L val brokerId: String = this.brokers.head.config.brokerId.toString setBrokerConfigs(brokerId, newValue) @@ -411,9 +394,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDynamicGroupConfigChange(quorum: String): Unit = { + @Test + def testDynamicGroupConfigChange(): Unit = { val newSessionTimeoutMs = 50000 val consumerGroupId = "group-foo" val admin = createAdminClient() @@ -438,9 +420,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(newSessionTimeoutMs, groupConfig.consumerSessionTimeoutMs()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft+kip848")) - def testDynamicShareGroupConfigChange(quorum: String): Unit = { + @Test + def testDynamicShareGroupConfigChange(): Unit = { val newRecordLockDurationMs = 50000 val shareGroupId = "group-foo" val admin = createAdminClient() @@ -465,9 +446,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { assertEquals(newRecordLockDurationMs, groupConfig.shareRecordLockDurationMs) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterDefaultGroupConfig(quorum: String): Unit = { + @Test + def testIncrementalAlterDefaultGroupConfig(): Unit = { val admin = createAdminClient() try { val resource = new ConfigResource(ConfigResource.Type.GROUP, "") diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 8e2dec5f565..ed1f94c3193 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -36,8 +36,7 @@ import org.apache.kafka.common.utils.ByteUtils import org.apache.kafka.common.{TopicPartition, Uuid, requests} import org.apache.kafka.server.config.ServerLogConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import scala.jdk.CollectionConverters._ @@ -118,9 +117,8 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testProduceRequestWithNullClientId(quorum: String): Unit = { + @Test + def testProduceRequestWithNullClientId(): Unit = { val topic = "topic" val topicPartition = new TopicPartition(topic, 0) val correlationId = -1 @@ -166,27 +164,23 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode), "There should be no error") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testHeaderOnlyRequest(quorum: String): Unit = { + @Test + def testHeaderOnlyRequest(): Unit = { verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, 1)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testInvalidApiKeyRequest(quorum: String): Unit = { + @Test + def testInvalidApiKeyRequest(): Unit = { verifyDisconnect(requestHeaderBytes(-1, 0)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testInvalidApiVersionRequest(quorum: String): Unit = { + @Test + def testInvalidApiVersionRequest(): Unit = { verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, -1)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMalformedHeaderRequest(quorum: String): Unit = { + @Test + def testMalformedHeaderRequest(): Unit = { val serializedBytes = { // Only send apiKey and apiVersion val buffer = ByteBuffer.allocate( diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala index a446bc90360..63215defd8f 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestMaxBytesTest.scala @@ -25,9 +25,7 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.util.{Optional, Properties} import scala.jdk.CollectionConverters._ @@ -104,9 +102,8 @@ class FetchRequestMaxBytesTest extends BaseRequestTest { * Note that when a single batch is larger than FetchMaxBytes, it will be * returned in full even if this is larger than FetchMaxBytes. See KIP-74. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumeMultipleRecords(quorum: String): Unit = { + @Test + def testConsumeMultipleRecords(): Unit = { createTopics() expectNextRecords(IndexedSeq(messages(0), messages(1)), 0) diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala index f96b2ceca31..5f5c17f50e6 100644 --- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala @@ -26,8 +26,7 @@ import org.apache.kafka.common.serialization.StringSerializer import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.server.record.BrokerCompressionType import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.util import java.util.Optional @@ -41,9 +40,8 @@ import scala.util.Random */ class FetchRequestTest extends BaseFetchRequestTest { - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testBrokerRespectsPartitionsOrderAndSizeLimits(quorum: String): Unit = { + @Test + def testBrokerRespectsPartitionsOrderAndSizeLimits(): Unit = { initProducer() val messagesPerPartition = 9 @@ -144,9 +142,8 @@ class FetchRequestTest extends BaseFetchRequestTest { evaluateResponse4(fetchResponse4V12, 12) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchRequestV4WithReadCommitted(quorum: String): Unit = { + @Test + def testFetchRequestV4WithReadCommitted(): Unit = { initProducer() val maxPartitionBytes = 200 val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head @@ -163,9 +160,8 @@ class FetchRequestTest extends BaseFetchRequestTest { assertTrue(records(partitionData).map(_.sizeInBytes).sum > 0) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchRequestToNonReplica(quorum: String): Unit = { + @Test + def testFetchRequestToNonReplica(): Unit = { val topic = "topic" val partition = 0 val topicPartition = new TopicPartition(topic, partition) @@ -194,15 +190,13 @@ class FetchRequestTest extends BaseFetchRequestTest { assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, oldPartitionData.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testLastFetchedEpochValidation(quorum: String): Unit = { + @Test + def testLastFetchedEpochValidation(): Unit = { checkLastFetchedEpochValidation(ApiKeys.FETCH.latestVersion()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testLastFetchedEpochValidationV12(quorum: String): Unit = { + @Test + def testLastFetchedEpochValidationV12(): Unit = { checkLastFetchedEpochValidation(12) } @@ -249,15 +243,13 @@ class FetchRequestTest extends BaseFetchRequestTest { assertEquals(firstEpochEndOffset, divergingEpoch.endOffset) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCurrentEpochValidation(quorum: String): Unit = { + @Test + def testCurrentEpochValidation(): Unit = { checkCurrentEpochValidation(ApiKeys.FETCH.latestVersion()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCurrentEpochValidationV12(quorum: String): Unit = { + @Test + def testCurrentEpochValidationV12(): Unit = { checkCurrentEpochValidation(12) } @@ -299,15 +291,13 @@ class FetchRequestTest extends BaseFetchRequestTest { assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch - 1)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testEpochValidationWithinFetchSession(quorum: String): Unit = { + @Test + def testEpochValidationWithinFetchSession(): Unit = { checkEpochValidationWithinFetchSession(ApiKeys.FETCH.latestVersion()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testEpochValidationWithinFetchSessionV12(quorum: String): Unit = { + @Test + def testEpochValidationWithinFetchSessionV12(): Unit = { checkEpochValidationWithinFetchSession(12) } @@ -367,9 +357,8 @@ class FetchRequestTest extends BaseFetchRequestTest { * those partitions are returned in all incremental fetch requests. * This tests using FetchRequests that don't use topic IDs */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCreateIncrementalFetchWithPartitionsInErrorV12(quorum: String): Unit = { + @Test + def testCreateIncrementalFetchWithPartitionsInErrorV12(): Unit = { def createConsumerFetchRequest(topicPartitions: Seq[TopicPartition], metadata: JFetchMetadata, toForget: Seq[TopicIdPartition]): FetchRequest = @@ -430,9 +419,8 @@ class FetchRequestTest extends BaseFetchRequestTest { /** * Test that when a Fetch Request receives an unknown topic ID, it returns a top level error. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchWithPartitionsWithIdError(quorum: String): Unit = { + @Test + def testFetchWithPartitionsWithIdError(): Unit = { def createConsumerFetchRequest(fetchData: util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData], metadata: JFetchMetadata, toForget: Seq[TopicIdPartition]): FetchRequest = { @@ -475,9 +463,8 @@ class FetchRequestTest extends BaseFetchRequestTest { assertEquals(Errors.UNKNOWN_TOPIC_ID.code, responseData1.get(bar0).errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testZStdCompressedTopic(quorum: String): Unit = { + @Test + def testZStdCompressedTopic(): Unit = { // ZSTD compressed topic val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.ZSTD.name) val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head @@ -523,9 +510,8 @@ class FetchRequestTest extends BaseFetchRequestTest { assertEquals(3, records(data2).size) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testZStdCompressedRecords(quorum: String): Unit = { + @Test + def testZStdCompressedRecords(): Unit = { // Producer compressed topic val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.PRODUCER.name) val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala index 3d4ea198753..51c5d192c6d 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterExceptionHandlingTest.scala @@ -24,9 +24,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol import org.apache.kafka.server.config.QuotaConfig import org.apache.kafka.server.metrics.MetricConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.net.Socket import java.util.concurrent.atomic.AtomicInteger @@ -63,9 +61,8 @@ class KafkaMetricReporterExceptionHandlingTest extends BaseRequestTest { super.tearDown() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testBothReportersAreInvoked(quorum: String): Unit = { + @Test + def testBothReportersAreInvoked(): Unit = { val port = anySocketServer.boundPort(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)) val socket = new Socket("localhost", port) socket.setSoTimeout(10000) diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala index 23cc8509627..c8692661134 100644 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricsReporterTest.scala @@ -23,10 +23,8 @@ import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsRepo import org.apache.kafka.server.config.ServerConfigs import org.apache.kafka.server.metrics.MetricConfigs import org.apache.kafka.test.{TestUtils => JTestUtils} -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource object KafkaMetricsReporterTest { @@ -78,9 +76,8 @@ class KafkaMetricsReporterTest extends QuorumTestHarness { broker.startup() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testMetricsContextNamespacePresent(quorum: String): Unit = { + @Test + def testMetricsContextNamespacePresent(): Unit = { assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID.get()) assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.NODEID.get()) assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX.get()) diff --git a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala index 5ba6ef34603..7b86baa3f5f 100644 --- a/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListOffsetsRequestTest.scala @@ -24,8 +24,7 @@ import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import java.util.{Optional, Properties} import scala.collection.Seq @@ -43,9 +42,8 @@ class ListOffsetsRequestTest extends BaseRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testListOffsetsErrorCodes(quorum: String): Unit = { + @Test + def testListOffsetsErrorCodes(): Unit = { val targetTimes = List(new ListOffsetsTopic() .setName(topic) .setPartitions(List(new ListOffsetsPartition() @@ -108,9 +106,8 @@ class ListOffsetsRequestTest extends BaseRequestTest { assertResponseError(error, brokerId, request) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCurrentEpochValidation(quorum: String): Unit = { + @Test + def testCurrentEpochValidation(): Unit = { val topic = "topic" val topicPartition = new TopicPartition(topic, 0) val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3) @@ -168,9 +165,8 @@ class ListOffsetsRequestTest extends BaseRequestTest { (partitionData.offset, partitionData.leaderEpoch, partitionData.errorCode()) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testResponseIncludesLeaderEpoch(quorum: String): Unit = { + @Test + def testResponseIncludesLeaderEpoch(): Unit = { val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3) val firstLeaderId = partitionToLeader(partition.partition) @@ -209,9 +205,8 @@ class ListOffsetsRequestTest extends BaseRequestTest { assertEquals((9L, firstLeaderEpoch, Errors.NONE.code), fetchOffsetAndEpochWithError(secondLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, -1)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testResponseDefaultOffsetAndLeaderEpochForAllVersions(quorum: String): Unit = { + @Test + def testResponseDefaultOffsetAndLeaderEpochForAllVersions(): Unit = { val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3) val firstLeaderId = partitionToLeader(partition.partition) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 6b73e570999..fae2b32b86b 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -26,9 +26,7 @@ import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffset import org.apache.kafka.common.{IsolationLevel, TopicPartition} import org.apache.kafka.storage.internals.log.{LogStartOffsetIncrementReason, OffsetResultHolder, UnifiedLog} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.Timeout -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{Test, Timeout} import java.io.File import java.util.{Optional, Properties, Random} @@ -47,9 +45,8 @@ class LogOffsetTest extends BaseRequestTest { props.put("log.retention.check.interval.ms", (5 * 1000 * 60).toString) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testGetOffsetsForUnknownTopic(quorum: String): Unit = { + @Test + def testGetOffsetsForUnknownTopic(): Unit = { val topicPartition = new TopicPartition("foo", 0) val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED) .setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP).asJava).build(1) @@ -58,9 +55,8 @@ class LogOffsetTest extends BaseRequestTest { } @deprecated("ListOffsetsRequest V0", since = "") - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testGetOffsetsAfterDeleteRecords(quorum: String): Unit = { + @Test + def testGetOffsetsAfterDeleteRecords(): Unit = { val topic = "kafka-" val topicPartition = new TopicPartition(topic, 0) val log = createTopicAndGetLog(topic, topicPartition) @@ -84,9 +80,8 @@ class LogOffsetTest extends BaseRequestTest { assertEquals(20L, consumerOffset) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchOffsetByTimestampForMaxTimestampAfterTruncate(quorum: String): Unit = { + @Test + def testFetchOffsetByTimestampForMaxTimestampAfterTruncate(): Unit = { val topic = "kafka-" val topicPartition = new TopicPartition(topic, 0) val log = createTopicAndGetLog(topic, topicPartition) @@ -106,9 +101,8 @@ class LogOffsetTest extends BaseRequestTest { assertEquals(Optional.empty, log.fetchOffsetByTimestamp(ListOffsetsRequest.MAX_TIMESTAMP, Optional.empty).timestampAndOffsetOpt) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchOffsetByTimestampForMaxTimestampWithUnorderedTimestamps(quorum: String): Unit = { + @Test + def testFetchOffsetByTimestampForMaxTimestampWithUnorderedTimestamps(): Unit = { val topic = "kafka-" val topicPartition = new TopicPartition(topic, 0) val log = createTopicAndGetLog(topic, topicPartition) @@ -125,9 +119,8 @@ class LogOffsetTest extends BaseRequestTest { assertEquals(6L, maxTimestampOffset.get.timestamp) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testGetOffsetsBeforeLatestTime(quorum: String): Unit = { + @Test + def testGetOffsetsBeforeLatestTime(): Unit = { val topic = "kafka-" val topicPartition = new TopicPartition(topic, 0) val log = createTopicAndGetLog(topic, topicPartition) @@ -158,9 +151,8 @@ class LogOffsetTest extends BaseRequestTest { assertFalse(FetchResponse.recordsOrFail(fetchResponse.responseData(topicNames, ApiKeys.FETCH.latestVersion).get(topicPartition)).batches.iterator.hasNext) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testEmptyLogsGetOffsets(quorum: String): Unit = { + @Test + def testEmptyLogsGetOffsets(): Unit = { val random = new Random val topic = "kafka-" val topicPartition = new TopicPartition(topic, random.nextInt(10)) @@ -182,9 +174,8 @@ class LogOffsetTest extends BaseRequestTest { assertFalse(offsetChanged) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testFetchOffsetByTimestampForMaxTimestampWithEmptyLog(quorum: String): Unit = { + @Test + def testFetchOffsetByTimestampForMaxTimestampWithEmptyLog(): Unit = { val topic = "kafka-" val topicPartition = new TopicPartition(topic, 0) val log = createTopicAndGetLog(topic, topicPartition) @@ -195,9 +186,8 @@ class LogOffsetTest extends BaseRequestTest { assertEquals(new OffsetResultHolder(Optional.empty[FileRecords.TimestampAndOffset]()), log.fetchOffsetByTimestamp(ListOffsetsRequest.MAX_TIMESTAMP, Optional.empty)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testGetOffsetsBeforeEarliestTime(quorum: String): Unit = { + @Test + def testGetOffsetsBeforeEarliestTime(): Unit = { val random = new Random val topic = "kafka-" val topicPartition = new TopicPartition(topic, random.nextInt(3)) diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index f9970d2967a..a05cb9f030d 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -27,9 +27,7 @@ import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerialize import org.apache.kafka.server.config.ReplicationConfigs import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.io.File import java.util.Properties @@ -104,9 +102,8 @@ class LogRecoveryTest extends QuorumTestHarness { super.tearDown() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testHWCheckpointNoFailuresSingleLogSegment(quorum: String): Unit = { + @Test + def testHWCheckpointNoFailuresSingleLogSegment(): Unit = { val numMessages = 2L sendMessages(numMessages.toInt) @@ -122,9 +119,8 @@ class LogRecoveryTest extends QuorumTestHarness { assertEquals(numMessages, followerHW) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testHWCheckpointWithFailuresSingleLogSegment(quorum: String): Unit = { + @Test + def testHWCheckpointWithFailuresSingleLogSegment(): Unit = { var leader = getLeaderIdForPartition(servers, topicPartition) assertEquals(0L, hwFile1.read().getOrDefault(topicPartition, 0L)) @@ -183,9 +179,8 @@ class LogRecoveryTest extends QuorumTestHarness { assertEquals(hw, hwFile2.read().getOrDefault(topicPartition, 0L)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testHWCheckpointNoFailuresMultipleLogSegments(quorum: String): Unit = { + @Test + def testHWCheckpointNoFailuresMultipleLogSegments(): Unit = { sendMessages(20) val hw = 20L // give some time for follower 1 to record leader HW of 600 @@ -200,9 +195,8 @@ class LogRecoveryTest extends QuorumTestHarness { assertEquals(hw, followerHW) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testHWCheckpointWithFailuresMultipleLogSegments(quorum: String): Unit = { + @Test + def testHWCheckpointWithFailuresMultipleLogSegments(): Unit = { var leader = getLeaderIdForPartition(servers, topicPartition) sendMessages(2) diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala index 2b2250ff95d..181fd2f644c 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -27,9 +27,7 @@ import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} import org.apache.kafka.metadata.BrokerState import org.apache.kafka.test.TestUtils.isValidClusterId import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo} import scala.collection.Seq import scala.jdk.CollectionConverters._ @@ -41,24 +39,21 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { doSetup(testInfo, createOffsetsTopic = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testClusterIdWithRequestVersion1(quorum: String): Unit = { + @Test + def testClusterIdWithRequestVersion1(): Unit = { val v1MetadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort)) val v1ClusterId = v1MetadataResponse.clusterId assertNull(v1ClusterId, s"v1 clusterId should be null") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testClusterIdIsValid(quorum: String): Unit = { + @Test + def testClusterIdIsValid(): Unit = { val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(4.toShort)) isValidClusterId(metadataResponse.clusterId) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testRack(quorum: String): Unit = { + @Test + def testRack(): Unit = { val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(4.toShort)) // Validate rack matches what's set in generateConfigs() above metadataResponse.brokers.forEach { broker => @@ -66,9 +61,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIsInternal(quorum: String): Unit = { + @Test + def testIsInternal(): Unit = { val internalTopic = Topic.GROUP_METADATA_TOPIC_NAME val notInternalTopic = "notInternal" // create the topics @@ -88,9 +82,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(Set(internalTopic).asJava, metadataResponse.buildCluster().internalTopics) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testNoTopicsRequest(quorum: String): Unit = { + @Test + def testNoTopicsRequest(): Unit = { // create some topics createTopic("t1", 3, 2) createTopic("t2", 3, 2) @@ -100,9 +93,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertTrue(metadataResponse.topicMetadata.isEmpty, "Response should have no topics") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAutoTopicCreation(quorum: String): Unit = { + @Test + def testAutoTopicCreation(): Unit = { val topic1 = "t1" val topic2 = "t2" val topic3 = "t3" @@ -128,9 +120,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response3.errors.get(topic5)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAutoCreateTopicWithInvalidReplicationFactor(quorum: String): Unit = { + @Test + def testAutoCreateTopicWithInvalidReplicationFactor(): Unit = { // Shutdown all but one broker so that the number of brokers is less than the default replication factor brokers.tail.foreach(_.shutdown()) brokers.tail.foreach(_.awaitShutdown()) @@ -144,9 +135,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(0, topicMetadata.partitionMetadata.size) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAllTopicsRequest(quorum: String): Unit = { + @Test + def testAllTopicsRequest(): Unit = { // create some topics createTopic("t1", 3, 2) createTopic("t2", 3, 2) @@ -162,9 +152,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(2, metadataResponseV1.topicMetadata.size(), "V1 Response should have 2 (all) topics") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testTopicIdsInResponse(quorum: String): Unit = { + @Test + def testTopicIdsInResponse(): Unit = { val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1)) val topic1 = "topic1" val topic2 = "topic2" @@ -192,9 +181,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { /** * Preferred replica should be the first item in the replicas list */ - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testPreferredReplica(quorum: String): Unit = { + @Test + def testPreferredReplica(): Unit = { val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1)) createTopicWithAssignment("t1", replicaAssignment) // Test metadata on two different brokers to ensure that metadata propagation works correctly @@ -216,9 +204,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testReplicaDownResponse(quorum: String): Unit = { + @Test + def testReplicaDownResponse(): Unit = { val replicaDownTopic = "replicaDown" val replicaCount = 3 @@ -262,9 +249,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { assertEquals(replicaCount, v1PartitionMetadata.replicaIds.size, s"Response should have $replicaCount replicas") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIsrAfterBrokerShutDownAndJoinsBack(quorum: String): Unit = { + @Test + def testIsrAfterBrokerShutDownAndJoinsBack(): Unit = { def checkIsr[B <: KafkaBroker]( brokers: Seq[B], topic: String @@ -300,9 +286,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest { checkIsr(brokers, topic) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testAliveBrokersWithNoTopics(quorum: String): Unit = { + @Test + def testAliveBrokersWithNoTopics(): Unit = { def checkMetadata[B <: KafkaBroker]( brokers: Seq[B], expectedBrokersCount: Int diff --git a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala index fc06a9eeeb7..1b13674685d 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetsForLeaderEpochRequestTest.scala @@ -26,16 +26,14 @@ import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.Test import scala.jdk.CollectionConverters._ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest { - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testOffsetsForLeaderEpochErrorCodes(quorum: String): Unit = { + @Test + def testOffsetsForLeaderEpochErrorCodes(): Unit = { val topic = "topic" val partition = new TopicPartition(topic, 0) val epochs = offsetForLeaderTopicCollectionFor(partition, 0, RecordBatch.NO_PARTITION_LEADER_EPOCH) @@ -57,9 +55,8 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest { assertResponseError(Errors.NOT_LEADER_OR_FOLLOWER, nonReplica, request) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCurrentEpochValidation(quorum: String): Unit = { + @Test + def testCurrentEpochValidation(): Unit = { val topic = "topic" val topicPartition = new TopicPartition(topic, 0) val partitionToLeader = createTopic(topic, replicationFactor = 3) diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index fcc64b45929..57545c7ba2b 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -32,9 +32,9 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.apache.kafka.server.record.BrokerCompressionType import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics import org.junit.jupiter.api.Assertions._ +import org.junit.jupiter.api.Test import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, MethodSource} -import org.junit.jupiter.params.provider.ValueSource import java.util.concurrent.TimeUnit import scala.jdk.CollectionConverters._ @@ -47,9 +47,8 @@ class ProduceRequestTest extends BaseRequestTest { val metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testSimpleProduceRequest(quorum: String): Unit = { + @Test + def testSimpleProduceRequest(): Unit = { val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") def sendAndCheck(memoryRecords: MemoryRecords, expectedOffset: Long): Unit = { @@ -163,9 +162,8 @@ class ProduceRequestTest extends BaseRequestTest { assertEquals("One or more records have been rejected due to invalid timestamp", partitionProduceResponse.errorMessage) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testProduceToNonReplica(quorum: String): Unit = { + @Test + def testProduceToNonReplica(): Unit = { val topic = "topic" val partition = 0 @@ -212,9 +210,8 @@ class ProduceRequestTest extends BaseRequestTest { }.getOrElse(throw new AssertionError(s"No leader elected for topic $topic")) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCorruptLz4ProduceRequest(quorum: String): Unit = { + @Test + def testCorruptLz4ProduceRequest(): Unit = { val (partition, leader) = createTopicAndFindPartitionWithLeader("topic") val topicId = getTopicIds().get("topic").get val timestamp = 1000000 @@ -247,9 +244,8 @@ class ProduceRequestTest extends BaseRequestTest { assertTrue(TestUtils.meterCount(s"${BrokerTopicMetrics.INVALID_MESSAGE_CRC_RECORDS_PER_SEC}") > 0) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testZSTDProduceRequest(quorum: String): Unit = { + @Test + def testZSTDProduceRequest(): Unit = { val topic = "topic" val partition = 0 diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index d86cc54ca93..66b41c0aaf1 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -17,15 +17,13 @@ package kafka.server -import org.junit.jupiter.api.AfterEach +import org.junit.jupiter.api.{AfterEach, Test} import kafka.utils.TestUtils import TestUtils._ import kafka.api.IntegrationTestHarness import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.StringSerializer -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource class ReplicaFetchTest extends IntegrationTestHarness { val topic1 = "foo" @@ -39,9 +37,8 @@ class ReplicaFetchTest extends IntegrationTestHarness { override def brokerCount: Int = 2 - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testReplicaFetcherThread(quorum: String): Unit = { + @Test + def testReplicaFetcherThread(): Unit = { val partition = 0 val testMessageList1 = List("test1", "test2", "test3", "test4") val testMessageList2 = List("test5", "test6", "test7", "test8") diff --git a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala index ae1e0a1f5e8..42a5e8accc9 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicationQuotasTest.scala @@ -37,9 +37,7 @@ import org.apache.kafka.server.common.{Feature, MetadataVersion} import org.apache.kafka.server.config.QuotaConfig import org.apache.kafka.server.quota.QuotaType import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.AfterEach -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, Test} import scala.jdk.CollectionConverters._ import scala.util.Using @@ -67,15 +65,13 @@ class ReplicationQuotasTest extends QuorumTestHarness { super.tearDown() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldBootstrapTwoBrokersWithLeaderThrottle(quorum: String): Unit = { + @Test + def shouldBootstrapTwoBrokersWithLeaderThrottle(): Unit = { shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldBootstrapTwoBrokersWithFollowerThrottle(quorum: String): Unit = { + @Test + def shouldBootstrapTwoBrokersWithFollowerThrottle(): Unit = { shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(false) } @@ -194,9 +190,8 @@ class ReplicationQuotasTest extends QuorumTestHarness { def tp(partition: Int): TopicPartition = new TopicPartition(topic, partition) - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldThrottleOldSegments(quorum: String): Unit = { + @Test + def shouldThrottleOldSegments(): Unit = { /** * Simple test which ensures throttled replication works when the dataset spans many segments */ diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala index 08bcbeba670..6cc31568d1e 100644 --- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala +++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala @@ -43,9 +43,7 @@ import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, A import org.apache.kafka.server.config.{QuotaConfig, ServerConfigs} import org.apache.kafka.server.quota.QuotaType import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo} -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource +import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo} import java.net.InetAddress import java.util @@ -133,32 +131,28 @@ class RequestQuotaTest extends BaseRequestTest { finally super.tearDown() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testResponseThrottleTime(quorum: String): Unit = { + @Test + def testResponseThrottleTime(): Unit = { for (apiKey <- clientActions ++ clusterActionsWithThrottleForBroker) submitTest(apiKey, () => checkRequestThrottleTime(apiKey)) waitAndCheckResults() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testResponseThrottleTimeWhenBothProduceAndRequestQuotasViolated(quorum: String): Unit = { + @Test + def testResponseThrottleTimeWhenBothProduceAndRequestQuotasViolated(): Unit = { submitTest(ApiKeys.PRODUCE, () => checkSmallQuotaProducerRequestThrottleTime()) waitAndCheckResults() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testResponseThrottleTimeWhenBothFetchAndRequestQuotasViolated(quorum: String): Unit = { + @Test + def testResponseThrottleTimeWhenBothFetchAndRequestQuotasViolated(): Unit = { submitTest(ApiKeys.FETCH, () => checkSmallQuotaConsumerRequestThrottleTime()) waitAndCheckResults() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnthrottledClient(quorum: String): Unit = { + @Test + def testUnthrottledClient(): Unit = { for (apiKey <- clientActions) { submitTest(apiKey, () => checkUnthrottledClient(apiKey)) } @@ -166,9 +160,8 @@ class RequestQuotaTest extends BaseRequestTest { waitAndCheckResults() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testExemptRequestTime(quorum: String): Unit = { + @Test + def testExemptRequestTime(): Unit = { // Exclude `DESCRIBE_QUORUM`, maybe it shouldn't be a cluster action val actions = clusterActions -- clusterActionsWithThrottleForBroker -- RequestQuotaTest.Envelope -- RequestQuotaTest.ShareGroupState - ApiKeys.DESCRIBE_QUORUM for (apiKey <- actions) { @@ -178,9 +171,8 @@ class RequestQuotaTest extends BaseRequestTest { waitAndCheckResults() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testUnauthorizedThrottle(quorum: String): Unit = { + @Test + def testUnauthorizedThrottle(): Unit = { RequestQuotaTest.principal = RequestQuotaTest.UnauthorizedPrincipal val apiKeys = ApiKeys.brokerApis diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 39327bbeaf8..cdc15b6c2be 100644 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -29,11 +29,11 @@ import org.apache.kafka.common.utils.Exit import org.apache.kafka.metadata.BrokerState import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs} import org.apache.kafka.storage.internals.log.LogManager -import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout} +import org.junit.jupiter.api.{BeforeEach, Test, TestInfo, Timeout} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.function.Executable import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{MethodSource, ValueSource} +import org.junit.jupiter.params.provider.MethodSource import java.time.Duration import java.util.Properties @@ -134,18 +134,16 @@ class ServerShutdownTest extends KafkaServerTestHarness { producer.close() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testCleanShutdownAfterFailedStartup(quorum: String): Unit = { + @Test + def testCleanShutdownAfterFailedStartup(): Unit = { propsToChangeUponRestart.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "1000") shutdownBroker() shutdownKRaftController() verifyCleanShutdownAfterFailedStartup[CancellationException] } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testNoCleanShutdownAfterFailedStartupDueToCorruptLogs(quorum: String): Unit = { + @Test + def testNoCleanShutdownAfterFailedStartupDueToCorruptLogs(): Unit = { createTopic(topic) shutdownBroker() config.logDirs.foreach { dirName => @@ -174,9 +172,8 @@ class ServerShutdownTest extends KafkaServerTestHarness { } } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testShutdownWithKRaftControllerUnavailable(quorum: String): Unit = { + @Test + def testShutdownWithKRaftControllerUnavailable(): Unit = { shutdownKRaftController() killBroker(0, Duration.ofSeconds(1)) CoreUtils.delete(broker.config.logDirs) @@ -220,9 +217,8 @@ class ServerShutdownTest extends KafkaServerTestHarness { .count(isNonDaemonKafkaThread)) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsecutiveShutdown(quorum: String): Unit = { + @Test + def testConsecutiveShutdown(): Unit = { shutdownBroker() brokers.head.shutdown() } diff --git a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala index 2518da7a857..d678a497b53 100644 --- a/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/epoch/LeaderEpochIntegrationTest.scala @@ -34,10 +34,8 @@ import org.apache.kafka.common.serialization.StringSerializer import org.apache.kafka.common.utils.{LogContext, Time} import org.apache.kafka.server.network.BrokerEndPoint import org.apache.kafka.test.{TestUtils => JTestUtils} -import org.junit.jupiter.api.AfterEach +import org.junit.jupiter.api.{AfterEach, Test} import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource import scala.collection.mutable.ListBuffer import scala.collection.{Map, Seq} @@ -64,9 +62,8 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging { super.tearDown() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldAddCurrentLeaderEpochToMessagesAsTheyAreWrittenToLeader(quorum: String): Unit = { + @Test + def shouldAddCurrentLeaderEpochToMessagesAsTheyAreWrittenToLeader(): Unit = { brokers ++= (0 to 1).map { id => createBroker(fromProps(createBrokerConfig(id))) } // Given two topics with replication of a single partition @@ -97,9 +94,8 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging { waitUntilTrue(() => messagesHaveLeaderEpoch(brokers(0), expectedLeaderEpoch, 4), "Leader epoch should be 1") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldSendLeaderEpochRequestAndGetAResponse(quorum: String): Unit = { + @Test + def shouldSendLeaderEpochRequestAndGetAResponse(): Unit = { //3 brokers, put partition on 100/101 and then pretend to be 102 brokers ++= (100 to 102).map { id => createBroker(fromProps(createBrokerConfig(id))) } @@ -145,9 +141,8 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging { fetcher1.close() } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def shouldIncreaseLeaderEpochBetweenLeaderRestarts(quorum: String): Unit = { + @Test + def shouldIncreaseLeaderEpochBetweenLeaderRestarts(): Unit = { //Setup: we are only interested in the single partition on broker 101 brokers += createBroker(fromProps(createBrokerConfig(100))) assertEquals(controllerServer.config.nodeId, waitUntilQuorumLeaderElected(controllerServer)) diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java index 36073f26dcc..a8d47549778 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/AuthorizerIntegrationTest.java @@ -21,8 +21,7 @@ import kafka.api.AbstractAuthorizerIntegrationTest; import org.apache.kafka.common.acl.AccessControlEntry; import org.apache.kafka.common.errors.GroupIdNotFoundException; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.concurrent.ExecutionException; @@ -35,9 +34,8 @@ import static org.junit.jupiter.api.Assertions.assertInstanceOf; import static org.junit.jupiter.api.Assertions.fail; public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { - @ParameterizedTest - @ValueSource(strings = {"kraft"}) - public void testDescribeGroupCliWithGroupDescribe(String quorum) throws Exception { + @Test + public void testDescribeGroupCliWithGroupDescribe() throws Exception { addAndVerifyAcls(CollectionConverters.asScala(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), "*", DESCRIBE, ALLOW))).toSet(), groupResource()); String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};