From 31b362341dbb663cbd58decf137f8ac5b840da50 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 5 Sep 2024 16:20:44 -0700 Subject: [PATCH 1/7] KAFKA-17337: ConsumerConfig should default to CONSUMER for group.protocol Updated DEFAULT_GROUP_PROTOCOL to use CONSUMER instead of CLASSIC, updated GROUP_PROTOCOL_DOC to be a little more flexible going forward, and cleaned up unit tests that assumed CLASSIC as the default. --- .../apache/kafka/clients/consumer/ConsumerConfig.java | 9 +++++---- .../kafka/clients/consumer/ConsumerConfigTest.java | 5 +++-- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index e32d42861387e..ec3eb791b8f1c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -43,6 +43,7 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.CooperativeStickyAssignor.COOPERATIVE_STICKY_ASSIGNOR_NAME; import static org.apache.kafka.clients.consumer.RangeAssignor.RANGE_ASSIGNOR_NAME; @@ -110,10 +111,10 @@ public class ConsumerConfig extends AbstractConfig { * group.protocol */ public static final String GROUP_PROTOCOL_CONFIG = "group.protocol"; - public static final String DEFAULT_GROUP_PROTOCOL = GroupProtocol.CLASSIC.name().toLowerCase(Locale.ROOT); - public static final String GROUP_PROTOCOL_DOC = "The group protocol consumer should use. We currently " + - "support \"classic\" or \"consumer\". If \"consumer\" is specified, then the consumer group protocol will be " + - "used. Otherwise, the classic group protocol will be used."; + public static final String DEFAULT_GROUP_PROTOCOL = GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT); + public static final String GROUP_PROTOCOL_DOC = "The group protocol to use when communicating with the Kafka " + + "cluster. The supported group protocols are: " + + Arrays.stream(GroupProtocol.values()).map(gp -> gp.name().toLowerCase()).collect(Collectors.joining(", ")) + "."; /** * group.remote.assignor diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java index 99c45f05c1559..8f2c4c0aa8385 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerConfigTest.java @@ -170,12 +170,12 @@ public void testCaseInsensitiveSecurityProtocol() { } @Test - public void testDefaultConsumerGroupConfig() { + public void testDefaultGroupProtocol() { final Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); final ConsumerConfig consumerConfig = new ConsumerConfig(configs); - assertEquals("classic", consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); + assertEquals(ConsumerConfig.DEFAULT_GROUP_PROTOCOL, consumerConfig.getString(ConsumerConfig.GROUP_PROTOCOL_CONFIG)); assertNull(consumerConfig.getString(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG)); } @@ -199,6 +199,7 @@ public void testRemoteAssignorWithClassicGroupProtocol() { final Map configs = new HashMap<>(); configs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, keyDeserializerClass); configs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, valueDeserializerClass); + configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, GroupProtocol.CLASSIC.name()); configs.put(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG, remoteAssignorName); ConfigException exception = assertThrows(ConfigException.class, () -> new ConsumerConfig(configs)); assertTrue(exception.getMessage().contains(ConsumerConfig.GROUP_REMOTE_ASSIGNOR_CONFIG + " cannot be set when " + ConsumerConfig.GROUP_PROTOCOL_CONFIG + "=" + GroupProtocol.CLASSIC.name())); From 1b590f9f044b85054ef3269ec1d0a977919afb64 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 5 Sep 2024 17:28:11 -0700 Subject: [PATCH 2/7] Fixed issue with missing Locale in toLowerCase() --- .../java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index ec3eb791b8f1c..63cba5d53c543 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -114,7 +114,7 @@ public class ConsumerConfig extends AbstractConfig { public static final String DEFAULT_GROUP_PROTOCOL = GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT); public static final String GROUP_PROTOCOL_DOC = "The group protocol to use when communicating with the Kafka " + "cluster. The supported group protocols are: " + - Arrays.stream(GroupProtocol.values()).map(gp -> gp.name().toLowerCase()).collect(Collectors.joining(", ")) + "."; + Arrays.stream(GroupProtocol.values()).map(gp -> gp.name().toLowerCase(Locale.ROOT)).collect(Collectors.joining(", ")) + "."; /** * group.remote.assignor From 5684d131571cb2b88d47fd6a5e55bfbe6f3a389f Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 6 Sep 2024 14:45:17 -0700 Subject: [PATCH 3/7] Fixed StackOverflowError For some reason, listing all the GroupProtocol options dynamically led to a StackOverflowError. Changed it to list the protocols explicitly to avoid this. The system is out of resources. Consult the following stack trace for details. java.lang.StackOverflowError at com.sun.tools.javac.comp.Attr.visitLambda(Attr.java:2296) at com.sun.tools.javac.tree.JCTree$JCLambda.accept(JCTree.java:1624) at com.sun.tools.javac.comp.Attr.attribTree(Attr.java:576) at com.sun.tools.javac.comp.Attr.visitLambda(Attr.java:2435) at com.sun.tools.javac.tree.JCTree$JCLambda.accept(JCTree.java:1624) at com.sun.tools.javac.comp.Attr.attribTree(Attr.java:576) at com.sun.tools.javac.comp.Attr.visitLambda(Attr.java:2435) at com.sun.tools.javac.tree.JCTree$JCLambda.accept(JCTree.java:1624) at com.sun.tools.javac.comp.Attr.attribTree(Attr.java:576) at com.sun.tools.javac.comp.Attr.visitLambda(Attr.java:2435) at com.sun.tools.javac.tree.JCTree$JCLambda.accept(JCTree.java:1624) --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 63cba5d53c543..fdc6cc4a9136c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -114,7 +114,10 @@ public class ConsumerConfig extends AbstractConfig { public static final String DEFAULT_GROUP_PROTOCOL = GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT); public static final String GROUP_PROTOCOL_DOC = "The group protocol to use when communicating with the Kafka " + "cluster. The supported group protocols are: " + - Arrays.stream(GroupProtocol.values()).map(gp -> gp.name().toLowerCase(Locale.ROOT)).collect(Collectors.joining(", ")) + "."; + GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT) + + " and " + + GroupProtocol.CLASSIC.name().toLowerCase(Locale.ROOT) + + "."; /** * group.remote.assignor From 297e5dc780ccf5323df2e93bb81c61fe0835c162 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Fri, 6 Sep 2024 14:46:47 -0700 Subject: [PATCH 4/7] Update ConsumerConfig.java --- .../java/org/apache/kafka/clients/consumer/ConsumerConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index fdc6cc4a9136c..442f3e91f79c8 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -43,7 +43,6 @@ import java.util.Properties; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.CooperativeStickyAssignor.COOPERATIVE_STICKY_ASSIGNOR_NAME; import static org.apache.kafka.clients.consumer.RangeAssignor.RANGE_ASSIGNOR_NAME; From 1294d4b7d20910cda89056cf92dcf80107d19b53 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Wed, 2 Oct 2024 19:47:14 -0700 Subject: [PATCH 5/7] Updates to parameterize test methods to ensure compatible combinations --- .../kafka/api/AuthorizerIntegrationTest.scala | 673 +++++++++--------- .../integration/kafka/api/BaseQuotaTest.scala | 45 +- .../kafka/api/EndToEndAuthorizationTest.scala | 103 +-- 3 files changed, 432 insertions(+), 389 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index af886d6a51dfb..9636ae2517c96 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -18,7 +18,7 @@ import java.util import java.util.concurrent.ExecutionException import java.util.regex.Pattern import java.util.{Collections, Optional, Properties} -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import kafka.utils.TestUtils.waitUntilTrue import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, NewTopic} import org.apache.kafka.clients.consumer._ @@ -57,7 +57,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, ValueSource} +import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource} import java.util.Collections.singletonList import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic @@ -710,9 +710,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizationWithTopicExisting(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizationWithTopicExisting(quorum: String, groupProtocol: String): Unit = { //First create the topic so we have a valid topic ID sendRequests(mutable.Map(ApiKeys.CREATE_TOPICS -> createTopicsRequest)) @@ -768,9 +768,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizationWithTopicNotExisting(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizationWithTopicNotExisting(quorum: String, groupProtocol: String): Unit = { val id = Uuid.randomUuid() val topicNames = Map(id -> "topic") val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest]( @@ -794,8 +794,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } @ParameterizedTest - @CsvSource(value = Array("zk,false", "zk,true", "kraft,false", "kraft,true")) - def testTopicIdAuthorization(quorum: String, withTopicExisting: Boolean): Unit = { + @CsvSource(value = Array("zk,classic,false", "zk,classic,true", "kraft,classic,false", "kraft,consumer,false", "kraft,classic,true", "kraft,consumer,true")) + def testTopicIdAuthorization(quorum: String, groupProtocol: String, withTopicExisting: Boolean): Unit = { val topicId = if (withTopicExisting) { createTopicWithBrokerPrincipal(topic) getTopicIds()(topic) @@ -846,9 +846,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizationFetchV12WithTopicNotExisting(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizationFetchV12WithTopicNotExisting(quorum: String, groupProtocol: String): Unit = { val id = Uuid.ZERO_UUID val topicNames = Map(id -> "topic") val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest]( @@ -858,9 +858,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequests(requestKeyToRequest, false, topicNames) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreateTopicAuthorizationWithClusterCreate(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreateTopicAuthorizationWithClusterCreate(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() val resources = Set[ResourceType](TOPIC) @@ -871,9 +871,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(createTopicsRequest, resources, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testFetchFollowerRequest(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testFetchFollowerRequest(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val request = createFetchFollowerRequest @@ -891,9 +891,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val data = new IncrementalAlterConfigsRequestData @@ -915,9 +915,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetsForLeaderEpochClusterPermission(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetsForLeaderEpochClusterPermission(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val request = offsetsForLeaderEpochRequest @@ -934,50 +934,50 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testProduceWithNoTopicAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testProduceWithTopicDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceWithTopicDescribe(quorum: String, groupProtocol: String): 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("zk", "kraft")) - def testProduceWithTopicRead(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceWithTopicRead(quorum: String, groupProtocol: String): 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("zk", "kraft")) - def testProduceWithTopicWrite(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceWithTopicWrite(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) val producer = createProducer() sendRecords(producer, numRecords, tp) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreatePermissionOnTopicToWriteToNonExistentTopic(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreatePermissionOnTopicToWriteToNonExistentTopic(quorum: String, groupProtocol: String): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(TOPIC) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreatePermissionOnClusterToWriteToNonExistentTopic(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreatePermissionOnClusterToWriteToNonExistentTopic(quorum: String, groupProtocol: String): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(CLUSTER) } @@ -994,9 +994,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRecords(producer, numRecords, tp) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testConsumeUsingAssignWithNoAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testConsumeUsingAssignWithNoAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1009,9 +1009,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TopicAuthorizationException], () => consumeRecords(consumer)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testSimpleConsumeWithOffsetLookupAndNoGroupAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testSimpleConsumeWithOffsetLookupAndNoGroupAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1029,9 +1029,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(group, e.groupId()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testSimpleConsumeWithExplicitSeekAndNoGroupAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testSimpleConsumeWithExplicitSeekAndNoGroupAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1049,9 +1049,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumeRecords(consumer) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testConsumeWithoutTopicDescribeAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testConsumeWithoutTopicDescribeAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1067,9 +1067,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testConsumeWithTopicDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testConsumeWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1086,9 +1086,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testConsumeWithTopicWrite(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testConsumeWithTopicWrite(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1105,9 +1105,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testConsumeWithTopicAndGroupRead(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testConsumeWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1124,9 +1124,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } @nowarn("cat=deprecation") - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testPatternSubscriptionWithNoTopicAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @CsvSource(value = Array("zk,classic", "kraft,classic")) + def testPatternSubscriptionWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1142,9 +1142,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertTrue(consumer.subscription.isEmpty) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testPatternSubscriptionWithTopicDescribeOnlyAndGroupRead(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPatternSubscriptionWithTopicDescribeOnlyAndGroupRead(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1161,9 +1161,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } @nowarn("cat=deprecation") - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testPatternSubscriptionWithTopicAndGroupRead(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @CsvSource(value = Array("zk,classic", "kraft,classic")) + def testPatternSubscriptionWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1194,9 +1194,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } @nowarn("cat=deprecation") - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testPatternSubscriptionMatchingInternalTopic(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPatternSubscriptionMatchingInternalTopic(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1224,9 +1224,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1250,9 +1250,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(GROUP_METADATA_TOPIC_NAME), e.unauthorizedTopics()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testPatternSubscriptionNotMatchingInternalTopic(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPatternSubscriptionNotMatchingInternalTopic(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1269,17 +1269,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumeRecords(consumer) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreatePermissionOnTopicToReadFromNonExistentTopic(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreatePermissionOnTopicToReadFromNonExistentTopic(quorum: String, groupProtocol: String): Unit = { testCreatePermissionNeededToReadFromNonExistentTopic("newTopic", Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)), TOPIC) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreatePermissionOnClusterToReadFromNonExistentTopic(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreatePermissionOnClusterToReadFromNonExistentTopic(quorum: String, groupProtocol: String): Unit = { testCreatePermissionNeededToReadFromNonExistentTopic("newTopic", Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)), CLUSTER) @@ -1310,9 +1310,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }, "Partition metadata not propagated.") } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreatePermissionMetadataRequestAutoCreate(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreatePermissionMetadataRequestAutoCreate(quorum: String, groupProtocol: String): Unit = { val readAcls = topicReadAcl(topicResource) addAndVerifyAcls(readAcls, topicResource) brokers.foreach(b => assertEquals(None, b.metadataCache.getPartitionInfo(topic, 0))) @@ -1332,24 +1332,24 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCommitWithNoAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCommitWithNoAccess(quorum: String, groupProtocol: String): Unit = { val consumer = createConsumer() assertThrows(classOf[GroupAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCommitWithNoTopicAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCommitWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCommitWithTopicWrite(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCommitWithTopicWrite(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1358,9 +1358,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCommitWithTopicDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCommitWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1369,17 +1369,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCommitWithNoGroupAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCommitWithNoGroupAccess(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) val consumer = createConsumer() assertThrows(classOf[GroupAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCommitWithTopicAndGroupRead(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCommitWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) @@ -1387,17 +1387,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetFetchWithNoAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetFetchWithNoAccess(quorum: String, groupProtocol: String): Unit = { val consumer = createConsumer() consumer.assign(List(tp).asJava) assertThrows(classOf[TopicAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetFetchWithNoGroupAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetFetchWithNoGroupAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) val consumer = createConsumer() @@ -1405,18 +1405,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[GroupAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetFetchWithNoTopicAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetFetchWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) val consumer = createConsumer() consumer.assign(List(tp).asJava) assertThrows(classOf[TopicAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetFetchAllTopicPartitionsAuthorization(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetFetchAllTopicPartitionsAuthorization(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val offset = 15L @@ -1446,9 +1446,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(offset, offsetFetchResponse.partitionDataMap(group).get(tp).offset) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetFetchMultipleGroupsAuthorization(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetFetchMultipleGroupsAuthorization(quorum: String, groupProtocol: String): Unit = { val groups: Seq[String] = (1 to 5).map(i => s"group$i") val groupResources = groups.map(group => new ResourcePattern(GROUP, group, LITERAL)) val topics: Seq[String] = (1 to 3).map(i => s"topic$i") @@ -1602,9 +1602,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetFetchTopicDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetFetchTopicDescribe(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) @@ -1613,9 +1613,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.position(tp) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testOffsetFetchWithTopicAndGroupRead(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testOffsetFetchWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) @@ -1624,58 +1624,58 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.position(tp) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testMetadataWithNoTopicAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMetadataWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.partitionsFor(topic)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testMetadataWithTopicDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMetadataWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val consumer = createConsumer() consumer.partitionsFor(topic) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testListOffsetsWithNoTopicAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testListOffsetsWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.endOffsets(Set(tp).asJava)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testListOffsetsWithTopicDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testListOffsetsWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val consumer = createConsumer() consumer.endOffsets(Set(tp).asJava) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeGroupApiWithNoGroupAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDescribeGroupApiWithNoGroupAcl(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val result = createAdminClient().describeConsumerGroups(Seq(group).asJava) TestUtils.assertFutureExceptionTypeEquals(result.describedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeGroupApiWithGroupDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDescribeGroupApiWithGroupDescribe(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testListGroupApiWithAndWithoutListGroupAcls(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testListGroupApiWithAndWithoutListGroupAcls(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) // write some record to the topic @@ -1722,9 +1722,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { otherConsumer.close() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteGroupApiWithDeleteGroupAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteGroupApiWithDeleteGroupAcl(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1736,9 +1736,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { createAdminClient().deleteConsumerGroups(Seq(group).asJava).deletedGroups().get(group).get() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteGroupApiWithNoDeleteGroupAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteGroupApiWithNoDeleteGroupAcl(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1750,16 +1750,16 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteGroupApiWithNoDeleteGroupAcl2(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteGroupApiWithNoDeleteGroupAcl2(quorum: String, groupProtocol: String): Unit = { val result = createAdminClient().deleteConsumerGroups(Seq(group).asJava) TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteGroupOffsetsWithAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteGroupOffsetsWithAcl(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)), groupResource) @@ -1773,9 +1773,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertNull(result.partitionResult(tp).get()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteGroupOffsetsWithoutDeleteAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteGroupOffsetsWithoutDeleteAcl(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1788,9 +1788,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) // Create the consumer group addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1809,16 +1809,16 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { TestUtils.assertFutureExceptionTypeEquals(result.partitionResult(tp), classOf[TopicAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteGroupOffsetsWithNoAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteGroupOffsetsWithNoAcl(quorum: String, groupProtocol: String): Unit = { val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterGroupConfigsWithAlterAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testIncrementalAlterGroupConfigsWithAlterAcl(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(groupAlterConfigsAcl(groupResource), groupResource) val request = incrementalAlterGroupConfigsRequest @@ -1826,9 +1826,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterGroupConfigsWithOperationAll(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testIncrementalAlterGroupConfigsWithOperationAll(quorum: String, groupProtocol: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -1837,9 +1837,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testIncrementalAlterGroupConfigsWithoutAlterAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testIncrementalAlterGroupConfigsWithoutAlterAcl(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() val request = incrementalAlterGroupConfigsRequest @@ -1847,9 +1847,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupConfigsWithDescribeAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testDescribeGroupConfigsWithDescribeAcl(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(groupDescribeConfigsAcl(groupResource), groupResource) val request = describeGroupConfigsRequest @@ -1857,9 +1857,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupConfigsWithOperationAll(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testDescribeGroupConfigsWithOperationAll(quorum: String, groupProtocol: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -1868,9 +1868,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testDescribeGroupConfigsWithoutDescribeAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testDescribeGroupConfigsWithoutDescribeAcl(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() val request = describeGroupConfigsRequest @@ -1878,42 +1878,42 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testUnauthorizedDeleteTopicsWithoutDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testUnauthorizedDeleteTopicsWithoutDescribe(quorum: String, groupProtocol: String): Unit = { val deleteResponse = connectAndReceive[DeleteTopicsResponse](deleteTopicsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testUnauthorizedDeleteTopicsWithDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testUnauthorizedDeleteTopicsWithDescribe(quorum: String, groupProtocol: String): 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("zk", "kraft")) - def testDeleteTopicsWithWildCardAuth(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteTopicsWithWildCardAuth(quorum: String, groupProtocol: String): 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("zk", "kraft")) - def testUnauthorizedDeleteRecordsWithoutDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testUnauthorizedDeleteRecordsWithoutDescribe(quorum: String, groupProtocol: String): 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("zk", "kraft")) - def testUnauthorizedDeleteRecordsWithDescribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testUnauthorizedDeleteRecordsWithDescribe(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) @@ -1921,9 +1921,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { partitions.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDeleteRecordsWithWildCardAuth(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDeleteRecordsWithWildCardAuth(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)), new ResourcePattern(TOPIC, "*", LITERAL)) val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) @@ -1931,40 +1931,40 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { partitions.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testUnauthorizedCreatePartitions(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testUnauthorizedCreatePartitions(quorum: String, groupProtocol: String): Unit = { val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, createPartitionsResponse.data.results.asScala.head.errorCode) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreatePartitionsWithWildCardAuth(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreatePartitionsWithWildCardAuth(quorum: String, groupProtocol: String): 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("zk", "kraft")) - def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), transactionalIdResource) val producer = buildTransactionalProducer() assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(quorum: String, groupProtocol: String): Unit = { val producer = buildTransactionalProducer() assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testSendOffsetsWithNoConsumerGroupDescribeAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testSendOffsetsWithNoConsumerGroupDescribeAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CLUSTER_ACTION, ALLOW)), clusterResource) @@ -1978,9 +1978,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testSendOffsetsWithNoConsumerGroupWriteAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testSendOffsetsWithNoConsumerGroupWriteAccess(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -1993,9 +1993,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) assertIdempotentSendAuthorizationFailure() @@ -2032,9 +2032,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertClusterAuthFailure() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testIdempotentProducerNoIdempotentWriteAclInProduce(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testIdempotentProducerNoIdempotentWriteAclInProduce(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, IDEMPOTENT_WRITE, ALLOW)), clusterResource) @@ -2061,17 +2061,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertTrue(e.getCause.isInstanceOf[TopicAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def shouldInitTransactionsWhenAclSet(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def shouldInitTransactionsWhenAclSet(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) val producer = buildTransactionalProducer() producer.initTransactions() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2086,9 +2086,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Set(topic), e.unauthorizedTopics.asScala) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testTransactionalProducerTopicAuthorizationExceptionInCommit(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testTransactionalProducerTopicAuthorizationExceptionInCommit(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2104,9 +2104,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2119,9 +2119,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { JTestUtils.assertFutureThrows(future, classOf[TransactionalIdAuthorizationException]) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2134,9 +2134,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.commitTransaction()) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testListTransactionsAuthorization(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testListTransactionsAuthorization(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2168,9 +2168,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertListTransactionResult(expectedTransactionalIds = Set(transactionalId)) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def shouldNotIncludeUnauthorizedTopicsInDescribeTransactionsResponse(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def shouldNotIncludeUnauthorizedTopicsInDescribeTransactionsResponse(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2191,9 +2191,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(List.empty, transactionStateData.topics.asScala.toList) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2211,9 +2211,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { producer.abortTransaction() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), groupResource) val producer = buildTransactionalProducer() @@ -2227,9 +2227,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, IDEMPOTENT_WRITE, ALLOW)), clusterResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2238,18 +2238,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } // Verify that metadata request without topics works without any ACLs and returns cluster id - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testClusterId(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testClusterId(quorum: String, groupProtocol: String): 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("zk", "kraft")) - def testRetryProducerInitializationAfterPermissionFix(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testRetryProducerInitializationAfterPermissionFix(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val wildcard = new ResourcePattern(TOPIC, ResourcePattern.WILDCARD_RESOURCE, LITERAL) val prefixed = new ResourcePattern(TOPIC, "t", PREFIXED) @@ -2271,9 +2271,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { producer.close() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizeByResourceTypeMultipleAddAndRemove(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizeByResourceTypeMultipleAddAndRemove(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) for (_ <- 1 to 3) { @@ -2289,9 +2289,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizeByResourceTypeIsolationUnrelatedDenyWontDominateAllow(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizeByResourceTypeIsolationUnrelatedDenyWontDominateAllow(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) createTopicWithBrokerPrincipal("topic-2") createTopicWithBrokerPrincipal("to") @@ -2312,9 +2312,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendSuccess() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizeByResourceTypeDenyTakesPrecedence(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizeByResourceTypeDenyTakesPrecedence(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val allowWriteAce = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW) addAndVerifyAcls(Set(allowWriteAce), topicResource) @@ -2325,9 +2325,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizeByResourceTypeWildcardResourceDenyDominate(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizeByResourceTypeWildcardResourceDenyDominate(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val wildcard = new ResourcePattern(TOPIC, ResourcePattern.WILDCARD_RESOURCE, LITERAL) val prefixed = new ResourcePattern(TOPIC, "t", PREFIXED) @@ -2343,9 +2343,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testAuthorizeByResourceTypePrefixedResourceDenyDominate(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testAuthorizeByResourceTypePrefixedResourceDenyDominate(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) val prefixed = new ResourcePattern(TOPIC, topic.substring(0, 1), PREFIXED) val literal = new ResourcePattern(TOPIC, topic, LITERAL) @@ -2357,9 +2357,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() // MetadataRequest versions older than 1 are not supported. @@ -2368,9 +2368,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testMetadataClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testMetadataClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) @@ -2389,9 +2389,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeTopicAclWithOperationAll(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDescribeTopicAclWithOperationAll(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2414,9 +2414,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Errors.NONE, topicResponse.error) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeTopicConfigsAclWithOperationAll(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDescribeTopicConfigsAclWithOperationAll(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2454,9 +2454,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() for (version <- ApiKeys.DESCRIBE_CLUSTER.oldestVersion to ApiKeys.DESCRIBE_CLUSTER.latestVersion) { @@ -2464,9 +2464,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testDescribeClusterClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testDescribeClusterClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) @@ -2484,9 +2484,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testHostAddressBasedAcls(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testHostAddressBasedAcls(quorum: String, groupProtocol: String): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2517,18 +2517,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testCreateAndCloseConsumerWithNoAccess(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testCreateAndCloseConsumerWithNoAccess(quorum: String, groupProtocol: String): Unit = { val consumer = createConsumer() val closeConsumer: Executable = () => consumer.close() // Close consumer without consuming anything. close() call should pass successfully and throw no exception. assertDoesNotThrow(closeConsumer, "Exception not expected on closing consumer") } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithReadAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testConsumerGroupHeartbeatWithReadAcl(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(groupReadAcl(groupResource), groupResource) val request = consumerGroupHeartbeatRequest @@ -2536,9 +2536,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithOperationAll(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testConsumerGroupHeartbeatWithOperationAll(quorum: String, groupProtocol: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2547,9 +2547,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupHeartbeatWithoutReadAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testConsumerGroupHeartbeatWithoutReadAcl(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() val request = consumerGroupHeartbeatRequest @@ -2557,9 +2557,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithDescribeAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testConsumerGroupDescribeWithDescribeAcl(quorum: String, groupProtocol: String): Unit = { addAndVerifyAcls(groupDescribeAcl(groupResource), groupResource) val request = consumerGroupDescribeRequest @@ -2567,9 +2567,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithOperationAll(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testConsumerGroupDescribeWithOperationAll(quorum: String, groupProtocol: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2578,9 +2578,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest - @ValueSource(strings = Array("kraft")) - def testConsumerGroupDescribeWithoutDescribeAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) + def testConsumerGroupDescribeWithoutDescribeAcl(quorum: String, groupProtocol: String): Unit = { removeAllClientAcls() val request = consumerGroupDescribeRequest @@ -2715,9 +2715,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testPrefixAcls(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testPrefixAcls(quorum: String, groupProtocol: String): 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)), @@ -2728,3 +2728,20 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { singletonList(new NewTopic("foobar", 1, 1.toShort))).all().get() } } +object AuthorizerIntegrationTest { + + def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { + util.Arrays.stream(Array( + Arguments.of("zk", "classic"), + Arguments.of("kraft", "classic"), + Arguments.of("kraft", "consumer") + )) + } + + def getKraftQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { + util.Arrays.stream(Array( + Arguments.of("kraft", "classic"), + Arguments.of("kraft", "consumer") + )) + } +} diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 5f9f9aac36199..7b5aba0de21ec 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -21,7 +21,7 @@ import java.util.{Collections, Properties} import com.yammer.metrics.core.{Histogram, Meter} import kafka.api.QuotaTestClients._ import kafka.server.{ClientQuotaManager, KafkaBroker} -import kafka.utils.TestUtils +import kafka.utils.{TestInfoUtils, TestUtils} import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} import org.apache.kafka.clients.producer._ @@ -39,7 +39,7 @@ import org.apache.kafka.server.quota.QuotaType 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.params.provider.{Arguments, MethodSource} import scala.collection.Map import scala.jdk.CollectionConverters._ @@ -89,9 +89,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients = createQuotaTestClients(topic1, leaderNode) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testThrottledProducerConsumer(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testThrottledProducerConsumer(quorum: String, groupProtocol: String): Unit = { val numRecords = 1000 val produced = quotaTestClients.produceUntilThrottled(numRecords) quotaTestClients.verifyProduceThrottle(expectThrottle = true) @@ -101,9 +101,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = true) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testProducerConsumerOverrideUnthrottled(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProducerConsumerOverrideUnthrottled(quorum: String, groupProtocol: String): Unit = { // Give effectively unlimited quota for producer and consumer val props = new Properties() props.put(QuotaConfigs.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, Long.MaxValue.toString) @@ -121,9 +121,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = false) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testProducerConsumerOverrideLowerQuota(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProducerConsumerOverrideLowerQuota(quorum: String, groupProtocol: String): Unit = { // consumer quota is set such that consumer quota * default quota window (10 seconds) is less than // MAX_PARTITION_FETCH_BYTES_CONFIG, so that we can test consumer ability to fetch in this case // In this case, 250 * 10 < 4096 @@ -139,9 +139,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = true) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testQuotaOverrideDelete(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testQuotaOverrideDelete(quorum: String, groupProtocol: String): Unit = { // Override producer and consumer quotas to unlimited quotaTestClients.overrideQuotas(Long.MaxValue, Long.MaxValue, Long.MaxValue.toDouble) quotaTestClients.waitForQuotaUpdate(Long.MaxValue, Long.MaxValue, Long.MaxValue.toDouble) @@ -166,9 +166,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = true) } - @ParameterizedTest - @ValueSource(strings = Array("zk", "kraft")) - def testThrottledRequest(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testThrottledRequest(quorum: String, groupProtocol: String): Unit = { quotaTestClients.overrideQuotas(Long.MaxValue, Long.MaxValue, 0.1) quotaTestClients.waitForQuotaUpdate(Long.MaxValue, Long.MaxValue, 0.1) @@ -396,3 +396,14 @@ abstract class QuotaTestClients(topic: String, } } } + +object BaseQuotaTest { + + def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { + util.Arrays.stream(Array( + Arguments.of("zk", "classic"), + Arguments.of("kraft", "classic"), + Arguments.of("kraft", "consumer") + )) + } +} diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index d32c5b74cc94b..121d441bb59e6 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -39,13 +39,14 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.security.auth._ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST -import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{CsvSource, ValueSource} +import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource} +import java.util import scala.jdk.CollectionConverters._ /** @@ -184,9 +185,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas /** * Tests the ability of producing and consuming with the appropriate ACLs set. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testProduceConsumeViaAssign(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceConsumeViaAssign(quorum: String, groupProtocol: String): Unit = { setAclsAndProduce(tp) val consumer = createConsumer() consumer.assign(List(tp).asJava) @@ -213,9 +214,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas ._2.asInstanceOf[Gauge[Double]] } - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testProduceConsumeViaSubscribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceConsumeViaSubscribe(quorum: String, groupProtocol: String): Unit = { setAclsAndProduce(tp) val consumer = createConsumer() consumer.subscribe(List(topic).asJava) @@ -223,9 +224,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testProduceConsumeWithWildcardAcls(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceConsumeWithWildcardAcls(quorum: String, groupProtocol: String): Unit = { setWildcardResourceAcls() val producer = createProducer() sendRecords(producer, numRecords, tp) @@ -235,9 +236,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testProduceConsumeWithPrefixedAcls(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceConsumeWithPrefixedAcls(quorum: String, groupProtocol: String): Unit = { setPrefixedResourceAcls() val producer = createProducer() sendRecords(producer, numRecords, tp) @@ -247,9 +248,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testProduceConsumeTopicAutoCreateTopicCreateAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testProduceConsumeTopicAutoCreateTopicCreateAcl(quorum: String, groupProtocol: String): Unit = { // topic2 is not created on setup() val tp2 = new TopicPartition("topic2", 0) setAclsAndProduce(tp2) @@ -315,14 +316,16 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas * messages and describe topics respectively when the describe ACL isn't set. * Also verifies that subsequent publish, consume and describe to authorized topic succeeds. */ - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @CsvSource(value = Array( - "kraft, true", - "kraft, false", - "zk, true", - "zk, false" + "kraft,classic,true", + "kraft,consumer,true", + "kraft,classic,false", + "kraft,consumer,false", + "zk,classic,true", + "zk,classic,false" )) - def testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl(quorum:String, isIdempotenceEnabled:Boolean): Unit = { + def testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl(quorum: String, groupProtocol: String, isIdempotenceEnabled: Boolean): Unit = { // Set consumer group acls since we are testing topic authorization setConsumerGroupAcls() @@ -386,14 +389,16 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas assertEquals(1, describeResults2.get(topic2).get().partitions().size()) } - @ParameterizedTest + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) @CsvSource(value = Array( - "kraft, true", - "kraft, false", - "zk, true", - "zk, false" + "kraft,classic,true", + "kraft,consumer,true", + "kraft,classic,false", + "kraft,consumer,false", + "zk,classic,true", + "zk,classic,false" )) - def testNoProduceWithDescribeAcl(quorum:String, isIdempotenceEnabled:Boolean): Unit = { + def testNoProduceWithDescribeAcl(quorum: String, groupProtocol: String, isIdempotenceEnabled: Boolean): Unit = { val superuserAdminClient = createSuperuserAdminClient() superuserAdminClient.createAcls(List(AclTopicDescribe()).asJava).values @@ -419,9 +424,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas * Tests that a consumer fails to consume messages without the appropriate * ACL set. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testNoConsumeWithoutDescribeAclViaAssign(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testNoConsumeWithoutDescribeAclViaAssign(quorum: String, groupProtocol: String): Unit = { noConsumeWithoutDescribeAclSetup() val consumer = createConsumer() consumer.assign(List(tp).asJava) @@ -430,9 +435,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testNoConsumeWithoutDescribeAclViaSubscribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testNoConsumeWithoutDescribeAclViaSubscribe(quorum: String, groupProtocol: String): Unit = { noConsumeWithoutDescribeAclSetup() val consumer = createConsumer() consumer.subscribe(List(topic).asJava) @@ -471,9 +476,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas } } - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testNoConsumeWithDescribeAclViaAssign(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testNoConsumeWithDescribeAclViaAssign(quorum: String, groupProtocol: String): Unit = { noConsumeWithDescribeAclSetup() val consumer = createConsumer() consumer.assign(List(tp).asJava) @@ -483,9 +488,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testNoConsumeWithDescribeAclViaSubscribe(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testNoConsumeWithDescribeAclViaSubscribe(quorum: String, groupProtocol: String): Unit = { noConsumeWithDescribeAclSetup() val consumer = createConsumer() consumer.subscribe(List(topic).asJava) @@ -512,9 +517,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas * Tests that a consumer fails to consume messages without the appropriate * ACL set. */ - @ParameterizedTest - @ValueSource(strings = Array("kraft", "zk")) - def testNoGroupAcl(quorum: String): Unit = { + @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) + def testNoGroupAcl(quorum: String, groupProtocol: String): Unit = { val superuserAdminClient = createSuperuserAdminClient() superuserAdminClient.createAcls(List(AclTopicWrite(), AclTopicCreate(), AclTopicDescribe()).asJava).values brokers.foreach { s => @@ -579,3 +584,13 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas } } +object EndToEndAuthorizationTest { + + def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { + util.Arrays.stream(Array( + Arguments.of("zk", "classic"), + Arguments.of("kraft", "classic"), + Arguments.of("kraft", "consumer") + )) + } +} From 26d1d303c8196300dbb9b7f023db17244d1ec516 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Tue, 29 Oct 2024 13:21:26 -0700 Subject: [PATCH 6/7] Reverting tests --- .../kafka/api/AuthorizerIntegrationTest.scala | 677 +++++++++--------- .../integration/kafka/api/BaseQuotaTest.scala | 45 +- .../kafka/api/EndToEndAuthorizationTest.scala | 99 ++- 3 files changed, 383 insertions(+), 438 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index a599fb0063aef..41208af7f2080 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -18,7 +18,7 @@ import java.util import java.util.concurrent.{ExecutionException, Semaphore} import java.util.regex.Pattern import java.util.{Collections, Optional, Properties} -import kafka.utils.{TestInfoUtils, TestUtils} +import kafka.utils.TestUtils import kafka.utils.TestUtils.waitUntilTrue import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, NewTopic} import org.apache.kafka.clients.consumer._ @@ -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.{Arguments, CsvSource, MethodSource} +import org.junit.jupiter.params.provider.{CsvSource, ValueSource} import java.util.Collections.singletonList import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic @@ -643,9 +643,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizationWithTopicExisting(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizationWithTopicExisting(quorum: String): Unit = { //First create the topic so we have a valid topic ID sendRequests(mutable.Map(ApiKeys.CREATE_TOPICS -> createTopicsRequest)) @@ -693,9 +693,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizationWithTopicNotExisting(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizationWithTopicNotExisting(quorum: String): Unit = { val id = Uuid.randomUuid() val topicNames = Map(id -> "topic") val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest]( @@ -719,8 +719,8 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } @ParameterizedTest - @CsvSource(value = Array("zk,classic,false", "zk,classic,true", "kraft,classic,false", "kraft,consumer,false", "kraft,classic,true", "kraft,consumer,true")) - def testTopicIdAuthorization(quorum: String, groupProtocol: String, withTopicExisting: Boolean): Unit = { + @CsvSource(value = Array("kraft,false", "kraft,true")) + def testTopicIdAuthorization(quorum: String, withTopicExisting: Boolean): Unit = { val topicId = if (withTopicExisting) { createTopicWithBrokerPrincipal(topic) getTopicIds()(topic) @@ -771,9 +771,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { /* * even if the topic doesn't exist, request APIs should not leak the topic name */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizationFetchV12WithTopicNotExisting(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizationFetchV12WithTopicNotExisting(quorum: String): Unit = { val id = Uuid.ZERO_UUID val topicNames = Map(id -> "topic") val requestKeyToRequest = mutable.LinkedHashMap[ApiKeys, AbstractRequest]( @@ -783,9 +783,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequests(requestKeyToRequest, false, topicNames) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreateTopicAuthorizationWithClusterCreate(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreateTopicAuthorizationWithClusterCreate(quorum: String): Unit = { removeAllClientAcls() val resources = Set[ResourceType](TOPIC) @@ -796,9 +796,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(createTopicsRequest, resources, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testFetchFollowerRequest(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testFetchFollowerRequest(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val request = createFetchFollowerRequest @@ -816,9 +816,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testIncrementalAlterConfigsRequestRequiresClusterPermissionForBrokerLogger(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val data = new IncrementalAlterConfigsRequestData @@ -840,9 +840,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetsForLeaderEpochClusterPermission(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetsForLeaderEpochClusterPermission(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val request = offsetsForLeaderEpochRequest @@ -859,50 +859,50 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resources, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceWithNoTopicAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceWithTopicRead(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceWithTopicRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) val producer = createProducer() assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords, tp)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceWithTopicWrite(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceWithTopicWrite(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) val producer = createProducer() sendRecords(producer, numRecords, tp) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreatePermissionOnTopicToWriteToNonExistentTopic(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreatePermissionOnTopicToWriteToNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(TOPIC) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreatePermissionOnClusterToWriteToNonExistentTopic(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreatePermissionOnClusterToWriteToNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToWriteToNonExistentTopic(CLUSTER) } @@ -919,9 +919,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRecords(producer, numRecords, tp) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testConsumeUsingAssignWithNoAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumeUsingAssignWithNoAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -934,9 +934,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TopicAuthorizationException], () => consumeRecords(consumer)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testSimpleConsumeWithOffsetLookupAndNoGroupAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testSimpleConsumeWithOffsetLookupAndNoGroupAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -954,9 +954,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(group, e.groupId()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testSimpleConsumeWithExplicitSeekAndNoGroupAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testSimpleConsumeWithExplicitSeekAndNoGroupAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -974,9 +974,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumeRecords(consumer) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testConsumeWithoutTopicDescribeAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumeWithoutTopicDescribeAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -992,9 +992,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testConsumeWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumeWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1011,9 +1011,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testConsumeWithTopicWrite(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumeWithTopicWrite(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1030,9 +1030,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testConsumeWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumeWithTopicAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1048,17 +1048,10 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumeRecords(consumer) } -<<<<<<< HEAD - @nowarn("cat=deprecation") - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @CsvSource(value = Array("zk,classic", "kraft,classic")) - def testPatternSubscriptionWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { -======= @ParameterizedTest @ValueSource(strings = Array("kraft")) def testPatternSubscriptionWithNoTopicAccess(quorum: String): Unit = { val assignSemaphore = new Semaphore(0) ->>>>>>> trunk createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1082,9 +1075,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertTrue(consumer.subscription.isEmpty) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testPatternSubscriptionWithTopicDescribeOnlyAndGroupRead(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testPatternSubscriptionWithTopicDescribeOnlyAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1100,10 +1093,10 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(topic), e.unauthorizedTopics()) } - @nowarn("cat=deprecation") - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @CsvSource(value = Array("zk,classic", "kraft,classic")) - def testPatternSubscriptionWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testPatternSubscriptionWithTopicAndGroupRead(quorum: String): Unit = { + val assignSemaphore = new Semaphore(0) createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1141,10 +1134,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertTrue(consumer.assignment().isEmpty) } - @nowarn("cat=deprecation") - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testPatternSubscriptionMatchingInternalTopic(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testPatternSubscriptionMatchingInternalTopic(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1172,9 +1164,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1198,9 +1190,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Collections.singleton(GROUP_METADATA_TOPIC_NAME), e.unauthorizedTopics()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testPatternSubscriptionNotMatchingInternalTopic(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testPatternSubscriptionNotMatchingInternalTopic(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -1217,17 +1209,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumeRecords(consumer) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreatePermissionOnTopicToReadFromNonExistentTopic(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreatePermissionOnTopicToReadFromNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToReadFromNonExistentTopic("newTopic", Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)), TOPIC) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreatePermissionOnClusterToReadFromNonExistentTopic(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreatePermissionOnClusterToReadFromNonExistentTopic(quorum: String): Unit = { testCreatePermissionNeededToReadFromNonExistentTopic("newTopic", Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)), CLUSTER) @@ -1258,9 +1250,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }, "Partition metadata not propagated.") } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreatePermissionMetadataRequestAutoCreate(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreatePermissionMetadataRequestAutoCreate(quorum: String): Unit = { val readAcls = topicReadAcl(topicResource) addAndVerifyAcls(readAcls, topicResource) brokers.foreach(b => assertEquals(None, b.metadataCache.getPartitionInfo(topic, 0))) @@ -1280,24 +1272,24 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCommitWithNoAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCommitWithNoAccess(quorum: String): Unit = { val consumer = createConsumer() assertThrows(classOf[GroupAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCommitWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCommitWithNoTopicAccess(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCommitWithTopicWrite(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCommitWithTopicWrite(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1306,9 +1298,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCommitWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCommitWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1317,17 +1309,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCommitWithNoGroupAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCommitWithNoGroupAccess(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) val consumer = createConsumer() assertThrows(classOf[GroupAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCommitWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCommitWithTopicAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) @@ -1335,17 +1327,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetFetchWithNoAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetFetchWithNoAccess(quorum: String): Unit = { val consumer = createConsumer() consumer.assign(List(tp).asJava) assertThrows(classOf[TopicAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetFetchWithNoGroupAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetFetchWithNoGroupAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) val consumer = createConsumer() @@ -1353,18 +1345,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[GroupAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetFetchWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetFetchWithNoTopicAccess(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) val consumer = createConsumer() consumer.assign(List(tp).asJava) assertThrows(classOf[TopicAuthorizationException], () => consumer.position(tp)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetFetchAllTopicPartitionsAuthorization(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetFetchAllTopicPartitionsAuthorization(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val offset = 15L @@ -1394,9 +1386,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(offset, offsetFetchResponse.partitionDataMap(group).get(tp).offset) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetFetchMultipleGroupsAuthorization(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetFetchMultipleGroupsAuthorization(quorum: String): Unit = { val groups: Seq[String] = (1 to 5).map(i => s"group$i") val groupResources = groups.map(group => new ResourcePattern(GROUP, group, LITERAL)) val topics: Seq[String] = (1 to 3).map(i => s"topic$i") @@ -1550,9 +1542,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetFetchTopicDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetFetchTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) @@ -1561,9 +1553,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.position(tp) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testOffsetFetchWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testOffsetFetchWithTopicAndGroupRead(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) @@ -1572,58 +1564,58 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { consumer.position(tp) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMetadataWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testMetadataWithNoTopicAccess(quorum: String): Unit = { val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.partitionsFor(topic)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMetadataWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testMetadataWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val consumer = createConsumer() consumer.partitionsFor(topic) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testListOffsetsWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testListOffsetsWithNoTopicAccess(quorum: String): Unit = { val consumer = createConsumer() assertThrows(classOf[TopicAuthorizationException], () => consumer.endOffsets(Set(tp).asJava)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testListOffsetsWithTopicDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testListOffsetsWithTopicDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val consumer = createConsumer() consumer.endOffsets(Set(tp).asJava) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupApiWithNoGroupAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeGroupApiWithNoGroupAcl(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val result = createAdminClient().describeConsumerGroups(Seq(group).asJava) TestUtils.assertFutureExceptionTypeEquals(result.describedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupApiWithGroupDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeGroupApiWithGroupDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), groupResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testListGroupApiWithAndWithoutListGroupAcls(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testListGroupApiWithAndWithoutListGroupAcls(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) // write some record to the topic @@ -1670,9 +1662,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { otherConsumer.close() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteGroupApiWithDeleteGroupAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteGroupApiWithDeleteGroupAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1684,9 +1676,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { createAdminClient().deleteConsumerGroups(Seq(group).asJava).deletedGroups().get(group).get() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteGroupApiWithNoDeleteGroupAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteGroupApiWithNoDeleteGroupAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1698,16 +1690,16 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteGroupApiWithNoDeleteGroupAcl2(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteGroupApiWithNoDeleteGroupAcl2(quorum: String): Unit = { val result = createAdminClient().deleteConsumerGroups(Seq(group).asJava) TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteGroupOffsetsWithAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteGroupOffsetsWithAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)), groupResource) @@ -1721,9 +1713,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertNull(result.partitionResult(tp).get()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteGroupOffsetsWithoutDeleteAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteGroupOffsetsWithoutDeleteAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1736,9 +1728,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) // Create the consumer group addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource) @@ -1757,16 +1749,16 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { TestUtils.assertFutureExceptionTypeEquals(result.partitionResult(tp), classOf[TopicAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteGroupOffsetsWithNoAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteGroupOffsetsWithNoAcl(quorum: String): Unit = { val result = createAdminClient().deleteConsumerGroupOffsets(group, Set(tp).asJava) TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testIncrementalAlterGroupConfigsWithAlterAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testIncrementalAlterGroupConfigsWithAlterAcl(quorum: String): Unit = { addAndVerifyAcls(groupAlterConfigsAcl(groupResource), groupResource) val request = incrementalAlterGroupConfigsRequest @@ -1774,9 +1766,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testIncrementalAlterGroupConfigsWithOperationAll(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testIncrementalAlterGroupConfigsWithOperationAll(quorum: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -1785,9 +1777,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testIncrementalAlterGroupConfigsWithoutAlterAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testIncrementalAlterGroupConfigsWithoutAlterAcl(quorum: String): Unit = { removeAllClientAcls() val request = incrementalAlterGroupConfigsRequest @@ -1795,9 +1787,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupConfigsWithDescribeAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeGroupConfigsWithDescribeAcl(quorum: String): Unit = { addAndVerifyAcls(groupDescribeConfigsAcl(groupResource), groupResource) val request = describeGroupConfigsRequest @@ -1805,9 +1797,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupConfigsWithOperationAll(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeGroupConfigsWithOperationAll(quorum: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -1816,9 +1808,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testDescribeGroupConfigsWithoutDescribeAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeGroupConfigsWithoutDescribeAcl(quorum: String): Unit = { removeAllClientAcls() val request = describeGroupConfigsRequest @@ -1826,42 +1818,42 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testUnauthorizedDeleteTopicsWithoutDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testUnauthorizedDeleteTopicsWithoutDescribe(quorum: String): Unit = { val deleteResponse = connectAndReceive[DeleteTopicsResponse](deleteTopicsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteResponse.data.responses.find(topic).errorCode) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testUnauthorizedDeleteTopicsWithDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testUnauthorizedDeleteTopicsWithDescribe(quorum: String): 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(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteTopicsWithWildCardAuth(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteTopicsWithWildCardAuth(quorum: String): 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(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testUnauthorizedDeleteRecordsWithoutDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testUnauthorizedDeleteRecordsWithoutDescribe(quorum: String): Unit = { val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, deleteRecordsResponse.data.topics.asScala.head. partitions.asScala.head.errorCode) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testUnauthorizedDeleteRecordsWithDescribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testUnauthorizedDeleteRecordsWithDescribe(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource) val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) @@ -1869,9 +1861,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { partitions.asScala.head.errorCode) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDeleteRecordsWithWildCardAuth(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDeleteRecordsWithWildCardAuth(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)), new ResourcePattern(TOPIC, "*", LITERAL)) val deleteRecordsResponse = connectAndReceive[DeleteRecordsResponse](deleteRecordsRequest) @@ -1879,40 +1871,40 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { partitions.asScala.head.errorCode) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testUnauthorizedCreatePartitions(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testUnauthorizedCreatePartitions(quorum: String): Unit = { val createPartitionsResponse = connectAndReceive[CreatePartitionsResponse](createPartitionsRequest) assertEquals(Errors.TOPIC_AUTHORIZATION_FAILED.code, createPartitionsResponse.data.results.asScala.head.errorCode) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreatePartitionsWithWildCardAuth(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreatePartitionsWithWildCardAuth(quorum: String): 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(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testTransactionalProducerInitTransactionsNoWriteTransactionalIdAcl(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), transactionalIdResource) val producer = buildTransactionalProducer() assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testTransactionalProducerInitTransactionsNoDescribeTransactionalIdAcl(quorum: String): Unit = { val producer = buildTransactionalProducer() assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.initTransactions()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testSendOffsetsWithNoConsumerGroupDescribeAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testSendOffsetsWithNoConsumerGroupDescribeAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CLUSTER_ACTION, ALLOW)), clusterResource) @@ -1926,9 +1918,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testSendOffsetsWithNoConsumerGroupWriteAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testSendOffsetsWithNoConsumerGroupWriteAccess(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -1941,9 +1933,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { () => producer.sendOffsetsToTransaction(Map(tp -> new OffsetAndMetadata(0L)).asJava, new ConsumerGroupMetadata(group))) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testIdempotentProducerNoIdempotentWriteAclInInitProducerId(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource) assertIdempotentSendAuthorizationFailure() @@ -1980,9 +1972,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertClusterAuthFailure() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testIdempotentProducerNoIdempotentWriteAclInProduce(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testIdempotentProducerNoIdempotentWriteAclInProduce(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, IDEMPOTENT_WRITE, ALLOW)), clusterResource) @@ -2009,17 +2001,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertTrue(e.getCause.isInstanceOf[TopicAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def shouldInitTransactionsWhenAclSet(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def shouldInitTransactionsWhenAclSet(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) val producer = buildTransactionalProducer() producer.initTransactions() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testTransactionalProducerTopicAuthorizationExceptionInSendCallback(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2034,9 +2026,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Set(topic), e.unauthorizedTopics.asScala) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testTransactionalProducerTopicAuthorizationExceptionInCommit(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testTransactionalProducerTopicAuthorizationExceptionInCommit(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2052,9 +2044,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessDuringSend(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2067,9 +2059,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { JTestUtils.assertFutureThrows(future, classOf[TransactionalIdAuthorizationException]) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnEndTransaction(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) @@ -2082,9 +2074,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertThrows(classOf[TransactionalIdAuthorizationException], () => producer.commitTransaction()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testListTransactionsAuthorization(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testListTransactionsAuthorization(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2116,9 +2108,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertListTransactionResult(expectedTransactionalIds = Set(transactionalId)) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def shouldNotIncludeUnauthorizedTopicsInDescribeTransactionsResponse(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def shouldNotIncludeUnauthorizedTopicsInDescribeTransactionsResponse(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2139,9 +2131,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(List.empty, transactionStateData.topics.asScala.toList) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def shouldSuccessfullyAbortTransactionAfterTopicAuthorizationException(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2159,9 +2151,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { producer.abortTransaction() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def shouldThrowTransactionalIdAuthorizationExceptionWhenNoTransactionAccessOnSendOffsetsToTxn(quorum: String): Unit = { addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), transactionalIdResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), groupResource) val producer = buildTransactionalProducer() @@ -2175,9 +2167,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { }) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def shouldSendSuccessfullyWhenIdempotentAndHasCorrectACL(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, IDEMPOTENT_WRITE, ALLOW)), clusterResource) addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource) @@ -2186,18 +2178,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } // Verify that metadata request without topics works without any ACLs and returns cluster id - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testClusterId(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testClusterId(quorum: String): 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(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testRetryProducerInitializationAfterPermissionFix(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testRetryProducerInitializationAfterPermissionFix(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val wildcard = new ResourcePattern(TOPIC, ResourcePattern.WILDCARD_RESOURCE, LITERAL) val prefixed = new ResourcePattern(TOPIC, "t", PREFIXED) @@ -2219,9 +2211,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { producer.close() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizeByResourceTypeMultipleAddAndRemove(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizeByResourceTypeMultipleAddAndRemove(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) for (_ <- 1 to 3) { @@ -2237,9 +2229,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizeByResourceTypeIsolationUnrelatedDenyWontDominateAllow(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizeByResourceTypeIsolationUnrelatedDenyWontDominateAllow(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) createTopicWithBrokerPrincipal("topic-2") createTopicWithBrokerPrincipal("to") @@ -2260,9 +2252,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendSuccess() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizeByResourceTypeDenyTakesPrecedence(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizeByResourceTypeDenyTakesPrecedence(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val allowWriteAce = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW) addAndVerifyAcls(Set(allowWriteAce), topicResource) @@ -2273,9 +2265,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizeByResourceTypeWildcardResourceDenyDominate(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizeByResourceTypeWildcardResourceDenyDominate(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val wildcard = new ResourcePattern(TOPIC, ResourcePattern.WILDCARD_RESOURCE, LITERAL) val prefixed = new ResourcePattern(TOPIC, "t", PREFIXED) @@ -2291,9 +2283,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testAuthorizeByResourceTypePrefixedResourceDenyDominate(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testAuthorizeByResourceTypePrefixedResourceDenyDominate(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) val prefixed = new ResourcePattern(TOPIC, topic.substring(0, 1), PREFIXED) val literal = new ResourcePattern(TOPIC, topic, LITERAL) @@ -2305,9 +2297,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertIdempotentSendAuthorizationFailure() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testMetadataClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { removeAllClientAcls() // MetadataRequest versions older than 1 are not supported. @@ -2316,9 +2308,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testMetadataClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testMetadataClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { removeAllClientAcls() val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) @@ -2337,9 +2329,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeTopicAclWithOperationAll(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeTopicAclWithOperationAll(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2362,9 +2354,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { assertEquals(Errors.NONE, topicResponse.error) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeTopicConfigsAclWithOperationAll(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeTopicConfigsAclWithOperationAll(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2402,9 +2394,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeClusterClusterAuthorizedOperationsWithoutDescribeCluster(quorum: String): Unit = { removeAllClientAcls() for (version <- ApiKeys.DESCRIBE_CLUSTER.oldestVersion to ApiKeys.DESCRIBE_CLUSTER.latestVersion) { @@ -2412,9 +2404,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testDescribeClusterClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testDescribeClusterClusterAuthorizedOperationsWithDescribeAndAlterCluster(quorum: String): Unit = { removeAllClientAcls() val clusterResource = new ResourcePattern(ResourceType.CLUSTER, Resource.CLUSTER_NAME, PatternType.LITERAL) @@ -2432,9 +2424,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testHostAddressBasedAcls(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testHostAddressBasedAcls(quorum: String): Unit = { createTopicWithBrokerPrincipal(topic) removeAllClientAcls() @@ -2465,18 +2457,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testCreateAndCloseConsumerWithNoAccess(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testCreateAndCloseConsumerWithNoAccess(quorum: String): Unit = { val consumer = createConsumer() val closeConsumer: Executable = () => consumer.close() // Close consumer without consuming anything. close() call should pass successfully and throw no exception. assertDoesNotThrow(closeConsumer, "Exception not expected on closing consumer") } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testConsumerGroupHeartbeatWithReadAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupHeartbeatWithReadAcl(quorum: String): Unit = { addAndVerifyAcls(groupReadAcl(groupResource), groupResource) val request = consumerGroupHeartbeatRequest @@ -2484,9 +2476,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testConsumerGroupHeartbeatWithOperationAll(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupHeartbeatWithOperationAll(quorum: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2495,9 +2487,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testConsumerGroupHeartbeatWithoutReadAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupHeartbeatWithoutReadAcl(quorum: String): Unit = { removeAllClientAcls() val request = consumerGroupHeartbeatRequest @@ -2505,9 +2497,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = false) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testConsumerGroupDescribeWithDescribeAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupDescribeWithDescribeAcl(quorum: String): Unit = { addAndVerifyAcls(groupDescribeAcl(groupResource), groupResource) val request = consumerGroupDescribeRequest @@ -2515,9 +2507,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testConsumerGroupDescribeWithOperationAll(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupDescribeWithOperationAll(quorum: String): Unit = { val allowAllOpsAcl = new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, ALL, ALLOW) addAndVerifyAcls(Set(allowAllOpsAcl), groupResource) @@ -2526,9 +2518,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { sendRequestAndVerifyResponseError(request, resource, isAuthorized = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getKraftQuorumAndGroupProtocolParametersAll")) - def testConsumerGroupDescribeWithoutDescribeAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testConsumerGroupDescribeWithoutDescribeAcl(quorum: String): Unit = { removeAllClientAcls() val request = consumerGroupDescribeRequest @@ -2663,9 +2655,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { ) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testPrefixAcls(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testPrefixAcls(quorum: String): 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)), @@ -2676,20 +2668,3 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest { singletonList(new NewTopic("foobar", 1, 1.toShort))).all().get() } } -object AuthorizerIntegrationTest { - - def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { - util.Arrays.stream(Array( - Arguments.of("zk", "classic"), - Arguments.of("kraft", "classic"), - Arguments.of("kraft", "consumer") - )) - } - - def getKraftQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { - util.Arrays.stream(Array( - Arguments.of("kraft", "classic"), - Arguments.of("kraft", "consumer") - )) - } -} diff --git a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala index 581b2316a15e3..859f489302387 100644 --- a/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseQuotaTest.scala @@ -21,7 +21,7 @@ import java.util.{Collections, Properties} import com.yammer.metrics.core.{Histogram, Meter} import kafka.api.QuotaTestClients._ import kafka.server.{ClientQuotaManager, KafkaBroker} -import kafka.utils.{TestInfoUtils, TestUtils} +import kafka.utils.TestUtils import org.apache.kafka.clients.admin.Admin import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig} import org.apache.kafka.clients.producer._ @@ -39,7 +39,7 @@ import org.apache.kafka.server.quota.QuotaType 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.{Arguments, MethodSource} +import org.junit.jupiter.params.provider.ValueSource import scala.collection.Map import scala.jdk.CollectionConverters._ @@ -89,9 +89,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients = createQuotaTestClients(topic1, leaderNode) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testThrottledProducerConsumer(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testThrottledProducerConsumer(quorum: String): Unit = { val numRecords = 1000 val produced = quotaTestClients.produceUntilThrottled(numRecords) quotaTestClients.verifyProduceThrottle(expectThrottle = true) @@ -101,9 +101,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProducerConsumerOverrideUnthrottled(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testProducerConsumerOverrideUnthrottled(quorum: String): Unit = { // Give effectively unlimited quota for producer and consumer val props = new Properties() props.put(QuotaConfig.PRODUCER_BYTE_RATE_OVERRIDE_CONFIG, Long.MaxValue.toString) @@ -121,9 +121,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = false) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProducerConsumerOverrideLowerQuota(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testProducerConsumerOverrideLowerQuota(quorum: String): Unit = { // consumer quota is set such that consumer quota * default quota window (10 seconds) is less than // MAX_PARTITION_FETCH_BYTES_CONFIG, so that we can test consumer ability to fetch in this case // In this case, 250 * 10 < 4096 @@ -139,9 +139,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testQuotaOverrideDelete(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testQuotaOverrideDelete(quorum: String): Unit = { // Override producer and consumer quotas to unlimited quotaTestClients.overrideQuotas(Long.MaxValue, Long.MaxValue, Long.MaxValue.toDouble) quotaTestClients.waitForQuotaUpdate(Long.MaxValue, Long.MaxValue, Long.MaxValue.toDouble) @@ -166,9 +166,9 @@ abstract class BaseQuotaTest extends IntegrationTestHarness { quotaTestClients.verifyConsumeThrottle(expectThrottle = true) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testThrottledRequest(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("zk", "kraft")) + def testThrottledRequest(quorum: String): Unit = { quotaTestClients.overrideQuotas(Long.MaxValue, Long.MaxValue, 0.1) quotaTestClients.waitForQuotaUpdate(Long.MaxValue, Long.MaxValue, 0.1) @@ -396,14 +396,3 @@ abstract class QuotaTestClients(topic: String, } } } - -object BaseQuotaTest { - - def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { - util.Arrays.stream(Array( - Arguments.of("zk", "classic"), - Arguments.of("kraft", "classic"), - Arguments.of("kraft", "consumer") - )) - } -} diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 5190f8531d9ba..ecfea5e0c02ca 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -38,14 +38,13 @@ import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED} import org.apache.kafka.common.security.auth._ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST -import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs, ZkConfigs} +import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs} import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout} import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource} +import org.junit.jupiter.params.provider.{CsvSource, ValueSource} -import java.util import scala.jdk.CollectionConverters._ /** @@ -171,9 +170,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas /** * Tests the ability of producing and consuming with the appropriate ACLs set. */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceConsumeViaAssign(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceConsumeViaAssign(quorum: String): Unit = { setAclsAndProduce(tp) val consumer = createConsumer() consumer.assign(List(tp).asJava) @@ -200,9 +199,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas ._2.asInstanceOf[Gauge[Double]] } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceConsumeViaSubscribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceConsumeViaSubscribe(quorum: String): Unit = { setAclsAndProduce(tp) val consumer = createConsumer() consumer.subscribe(List(topic).asJava) @@ -210,9 +209,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceConsumeWithWildcardAcls(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceConsumeWithWildcardAcls(quorum: String): Unit = { setWildcardResourceAcls() val producer = createProducer() sendRecords(producer, numRecords, tp) @@ -222,9 +221,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceConsumeWithPrefixedAcls(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceConsumeWithPrefixedAcls(quorum: String): Unit = { setPrefixedResourceAcls() val producer = createProducer() sendRecords(producer, numRecords, tp) @@ -234,9 +233,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testProduceConsumeTopicAutoCreateTopicCreateAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testProduceConsumeTopicAutoCreateTopicCreateAcl(quorum: String): Unit = { // topic2 is not created on setup() val tp2 = new TopicPartition("topic2", 0) setAclsAndProduce(tp2) @@ -302,16 +301,12 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas * messages and describe topics respectively when the describe ACL isn't set. * Also verifies that subsequent publish, consume and describe to authorized topic succeeds. */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @ParameterizedTest @CsvSource(value = Array( - "kraft,classic,true", - "kraft,consumer,true", - "kraft,classic,false", - "kraft,consumer,false", - "zk,classic,true", - "zk,classic,false" + "kraft, true", + "kraft, false", )) - def testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl(quorum: String, groupProtocol: String, isIdempotenceEnabled: Boolean): Unit = { + def testNoDescribeProduceOrConsumeWithoutTopicDescribeAcl(quorum:String, isIdempotenceEnabled:Boolean): Unit = { // Set consumer group acls since we are testing topic authorization setConsumerGroupAcls() @@ -375,16 +370,12 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas assertEquals(1, describeResults2.get(topic2).get().partitions().size()) } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) + @ParameterizedTest @CsvSource(value = Array( - "kraft,classic,true", - "kraft,consumer,true", - "kraft,classic,false", - "kraft,consumer,false", - "zk,classic,true", - "zk,classic,false" + "kraft, true", + "kraft, false", )) - def testNoProduceWithDescribeAcl(quorum: String, groupProtocol: String, isIdempotenceEnabled: Boolean): Unit = { + def testNoProduceWithDescribeAcl(quorum:String, isIdempotenceEnabled:Boolean): Unit = { val superuserAdminClient = createSuperuserAdminClient() superuserAdminClient.createAcls(List(AclTopicDescribe()).asJava).values @@ -410,9 +401,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas * Tests that a consumer fails to consume messages without the appropriate * ACL set. */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testNoConsumeWithoutDescribeAclViaAssign(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testNoConsumeWithoutDescribeAclViaAssign(quorum: String): Unit = { noConsumeWithoutDescribeAclSetup() val consumer = createConsumer() consumer.assign(List(tp).asJava) @@ -421,9 +412,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testNoConsumeWithoutDescribeAclViaSubscribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testNoConsumeWithoutDescribeAclViaSubscribe(quorum: String): Unit = { noConsumeWithoutDescribeAclSetup() val consumer = createConsumer() consumer.subscribe(List(topic).asJava) @@ -462,9 +453,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas } } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testNoConsumeWithDescribeAclViaAssign(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testNoConsumeWithDescribeAclViaAssign(quorum: String): Unit = { noConsumeWithDescribeAclSetup() val consumer = createConsumer() consumer.assign(List(tp).asJava) @@ -474,9 +465,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas confirmReauthenticationMetrics() } - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testNoConsumeWithDescribeAclViaSubscribe(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testNoConsumeWithDescribeAclViaSubscribe(quorum: String): Unit = { noConsumeWithDescribeAclSetup() val consumer = createConsumer() consumer.subscribe(List(topic).asJava) @@ -503,9 +494,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas * Tests that a consumer fails to consume messages without the appropriate * ACL set. */ - @ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames) - @MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll")) - def testNoGroupAcl(quorum: String, groupProtocol: String): Unit = { + @ParameterizedTest + @ValueSource(strings = Array("kraft")) + def testNoGroupAcl(quorum: String): Unit = { val superuserAdminClient = createSuperuserAdminClient() superuserAdminClient.createAcls(List(AclTopicWrite(), AclTopicCreate(), AclTopicDescribe()).asJava).values brokers.foreach { s => @@ -570,13 +561,3 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas } } -object EndToEndAuthorizationTest { - - def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = { - util.Arrays.stream(Array( - Arguments.of("zk", "classic"), - Arguments.of("kraft", "classic"), - Arguments.of("kraft", "consumer") - )) - } -} From ebd743431a795ab7a76c519817294152e9ec5217 Mon Sep 17 00:00:00 2001 From: Kirk True Date: Thu, 7 Nov 2024 15:56:33 -0800 Subject: [PATCH 7/7] Reverting refactoring that obfuscate the core change --- .../apache/kafka/clients/consumer/ConsumerConfig.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 9aa82bf3c2ae4..c84d0465d5514 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -112,12 +112,9 @@ public class ConsumerConfig extends AbstractConfig { */ public static final String GROUP_PROTOCOL_CONFIG = "group.protocol"; public static final String DEFAULT_GROUP_PROTOCOL = GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT); - public static final String GROUP_PROTOCOL_DOC = "The group protocol to use when communicating with the Kafka " + - "cluster. The supported group protocols are: " + - GroupProtocol.CONSUMER.name().toLowerCase(Locale.ROOT) + - " and " + - GroupProtocol.CLASSIC.name().toLowerCase(Locale.ROOT) + - "."; + public static final String GROUP_PROTOCOL_DOC = "The group protocol consumer should use. We currently " + + "support \"classic\" or \"consumer\". If \"consumer\" is specified, then the consumer group protocol will be " + + "used. Otherwise, the classic group protocol will be used."; /** * group.remote.assignor