KAFKA-17925 Convert Kafka Client integration tests to use KRaft (#17670)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
Kirk True 2024-11-09 18:38:01 -08:00 committed by GitHub
parent 0bc91be145
commit 42ea29c421
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
89 changed files with 981 additions and 946 deletions

View File

@ -66,7 +66,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testListMaxTimestampWithEmptyLog(quorum: String): Unit = {
val maxTimestampOffset = runFetchOffsets(adminClient, OffsetSpec.maxTimestamp(), topicName)
assertEquals(ListOffsetsResponse.UNKNOWN_OFFSET, maxTimestampOffset.offset())
@ -88,7 +88,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testThreeCompressedRecordsInOneBatch(quorum: String): Unit = {
produceMessagesInOneBatch("gzip")
verifyListOffsets()
@ -102,7 +102,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testThreeNonCompressedRecordsInOneBatch(quorum: String): Unit = {
produceMessagesInOneBatch()
verifyListOffsets()
@ -117,7 +117,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testThreeNonCompressedRecordsInSeparateBatch(quorum: String): Unit = {
produceMessagesInSeparateBatch()
verifyListOffsets()
@ -162,7 +162,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testThreeRecordsInOneBatchHavingDifferentCompressionTypeWithServer(quorum: String): Unit = {
val props: Properties = new Properties()
props.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4")
@ -172,7 +172,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testThreeRecordsInSeparateBatchHavingDifferentCompressionTypeWithServer(quorum: String): Unit = {
val props: Properties = new Properties()
props.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, "lz4")
@ -182,7 +182,7 @@ class ListOffsetsIntegrationTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testThreeCompressedRecordsInSeparateBatch(quorum: String): Unit = {
produceMessagesInSeparateBatch("gzip")
verifyListOffsets()

View File

@ -74,7 +74,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateRemoteTopicWithValidRetentionTime(quorum: String): Unit = {
val topicConfig = new Properties()
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@ -86,7 +86,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateRemoteTopicWithValidRetentionSize(quorum: String): Unit = {
val topicConfig = new Properties()
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@ -98,7 +98,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateRemoteTopicWithInheritedLocalRetentionTime(quorum: String): Unit = {
// inherited local retention ms is 1000
val topicConfig = new Properties()
@ -110,7 +110,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateRemoteTopicWithInheritedLocalRetentionSize(quorum: String): Unit = {
// inherited local retention bytes is 1024
val topicConfig = new Properties()
@ -122,7 +122,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateRemoteTopicWithInvalidRetentionTime(quorum: String): Unit = {
// inherited local retention ms is 1000
val topicConfig = new Properties()
@ -134,7 +134,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateRemoteTopicWithInvalidRetentionSize(quorum: String): Unit = {
// inherited local retention bytes is 1024
val topicConfig = new Properties()
@ -146,7 +146,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateCompactedRemoteStorage(quorum: String): Unit = {
val topicConfig = new Properties()
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@ -301,7 +301,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testEnableRemoteLogOnExistingTopicTest(quorum: String): Unit = {
val admin = createAdminClient()
val topicConfig = new Properties()
@ -319,7 +319,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testEnableRemoteLogWhenSystemRemoteStorageIsDisabled(quorum: String): Unit = {
val admin = createAdminClient()
@ -343,7 +343,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUpdateTopicConfigWithValidRetentionTimeTest(quorum: String): Unit = {
val admin = createAdminClient()
val topicConfig = new Properties()
@ -364,7 +364,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUpdateTopicConfigWithValidRetentionSizeTest(quorum: String): Unit = {
val admin = createAdminClient()
val topicConfig = new Properties()
@ -385,7 +385,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUpdateTopicConfigWithInheritedLocalRetentionTime(quorum: String): Unit = {
val admin = createAdminClient()
val topicConfig = new Properties()
@ -405,7 +405,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUpdateTopicConfigWithInheritedLocalRetentionSize(quorum: String): Unit = {
val admin = createAdminClient()
val topicConfig = new Properties()
@ -474,7 +474,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testTopicDeletion(quorum: String): Unit = {
MyRemoteStorageManager.deleteSegmentEventCounter.set(0)
val numPartitions = 2
@ -493,7 +493,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic(quorum: String): Unit = {
val topicConfig = new Properties()
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
@ -515,7 +515,7 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled(quorum: String): Unit = {
val topicConfig = new Properties()
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, false.toString)

View File

@ -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.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._
@ -53,7 +53,7 @@ import org.apache.kafka.security.authorizer.AclEntry
import org.apache.kafka.security.authorizer.AclEntry.WILDCARD_HOST
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
import org.junit.jupiter.params.provider.{CsvSource, MethodSource, ValueSource}
import java.util.Collections.singletonList
import org.apache.kafka.common.message.MetadataRequestData.MetadataRequestTopic
@ -919,9 +919,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
sendRecords(producer, numRecords, tp)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testConsumeUsingAssignWithNoAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testConsumeUsingAssignWithNoAccess(quorum: String, groupProtocol: 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
@ValueSource(strings = Array("kraft"))
def testSimpleConsumeWithOffsetLookupAndNoGroupAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testSimpleConsumeWithOffsetLookupAndNoGroupAccess(quorum: String, groupProtocol: 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
@ValueSource(strings = Array("kraft"))
def testSimpleConsumeWithExplicitSeekAndNoGroupAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testSimpleConsumeWithExplicitSeekAndNoGroupAccess(quorum: String, groupProtocol: 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
@ValueSource(strings = Array("kraft"))
def testConsumeWithoutTopicDescribeAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testConsumeWithoutTopicDescribeAccess(quorum: String, groupProtocol: 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
@ValueSource(strings = Array("kraft"))
def testConsumeWithTopicDescribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testConsumeWithTopicDescribe(quorum: String, groupProtocol: 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
@ValueSource(strings = Array("kraft"))
def testConsumeWithTopicWrite(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testConsumeWithTopicWrite(quorum: String, groupProtocol: 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
@ValueSource(strings = Array("kraft"))
def testConsumeWithTopicAndGroupRead(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testConsumeWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource)
@ -1048,9 +1048,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
consumeRecords(consumer)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testPatternSubscriptionWithNoTopicAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testPatternSubscriptionWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = {
val assignSemaphore = new Semaphore(0)
createTopicWithBrokerPrincipal(topic)
@ -1075,9 +1075,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertTrue(consumer.subscription.isEmpty)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testPatternSubscriptionWithTopicDescribeOnlyAndGroupRead(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testPatternSubscriptionWithTopicDescribeOnlyAndGroupRead(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource)
@ -1093,9 +1093,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertEquals(Collections.singleton(topic), e.unauthorizedTopics())
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testPatternSubscriptionWithTopicAndGroupRead(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testPatternSubscriptionWithTopicAndGroupRead(quorum: String, groupProtocol: String): Unit = {
val assignSemaphore = new Semaphore(0)
createTopicWithBrokerPrincipal(topic)
@ -1134,9 +1134,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertTrue(consumer.assignment().isEmpty)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testPatternSubscriptionMatchingInternalTopic(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testPatternSubscriptionMatchingInternalTopic(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource)
@ -1164,9 +1164,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
}
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testPatternSubscriptionMatchingInternalTopicWithDescribeOnlyPermission(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource)
@ -1190,9 +1190,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertEquals(Collections.singleton(GROUP_METADATA_TOPIC_NAME), e.unauthorizedTopics())
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testPatternSubscriptionNotMatchingInternalTopic(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testPatternSubscriptionNotMatchingInternalTopic(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, WRITE, ALLOW)), topicResource)
@ -1209,17 +1209,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
consumeRecords(consumer)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testCreatePermissionOnTopicToReadFromNonExistentTopic(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testCreatePermissionOnTopicToReadFromNonExistentTopic(quorum: String, groupProtocol: String): Unit = {
testCreatePermissionNeededToReadFromNonExistentTopic("newTopic",
Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)),
TOPIC)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testCreatePermissionOnClusterToReadFromNonExistentTopic(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testCreatePermissionOnClusterToReadFromNonExistentTopic(quorum: String, groupProtocol: String): Unit = {
testCreatePermissionNeededToReadFromNonExistentTopic("newTopic",
Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, CREATE, ALLOW)),
CLUSTER)
@ -1272,24 +1272,24 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
}
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testCommitWithNoAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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("kraft"))
def testCommitWithNoTopicAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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("kraft"))
def testCommitWithTopicWrite(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testCommitWithTopicWrite(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource)
@ -1298,9 +1298,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava))
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testCommitWithTopicDescribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testCommitWithTopicDescribe(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource)
@ -1309,17 +1309,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertThrows(classOf[TopicAuthorizationException], () => consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava))
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testCommitWithNoGroupAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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("kraft"))
def testCommitWithTopicAndGroupRead(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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)
@ -1327,17 +1327,17 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
consumer.commitSync(Map(tp -> new OffsetAndMetadata(5)).asJava)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testOffsetFetchWithNoAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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("kraft"))
def testOffsetFetchWithNoGroupAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testOffsetFetchWithNoGroupAccess(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), topicResource)
val consumer = createConsumer()
@ -1345,18 +1345,18 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertThrows(classOf[GroupAuthorizationException], () => consumer.position(tp))
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testOffsetFetchWithNoTopicAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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("kraft"))
def testOffsetFetchAllTopicPartitionsAuthorization(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testOffsetFetchAllTopicPartitionsAuthorization(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
val offset = 15L
@ -1386,9 +1386,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertEquals(offset, offsetFetchResponse.partitionDataMap(group).get(tp).offset)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testOffsetFetchMultipleGroupsAuthorization(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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")
@ -1542,9 +1542,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testOffsetFetchTopicDescribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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)
@ -1553,9 +1553,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
consumer.position(tp)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testOffsetFetchWithTopicAndGroupRead(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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)
@ -1564,32 +1564,32 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
consumer.position(tp)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testMetadataWithNoTopicAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testMetadataWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = {
val consumer = createConsumer()
assertThrows(classOf[TopicAuthorizationException], () => consumer.partitionsFor(topic))
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testMetadataWithTopicDescribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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("kraft"))
def testListOffsetsWithNoTopicAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testListOffsetsWithNoTopicAccess(quorum: String, groupProtocol: String): Unit = {
val consumer = createConsumer()
assertThrows(classOf[TopicAuthorizationException], () => consumer.endOffsets(Set(tp).asJava))
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testListOffsetsWithTopicDescribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testListOffsetsWithTopicDescribe(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DESCRIBE, ALLOW)), topicResource)
val consumer = createConsumer()
@ -1613,9 +1613,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
createAdminClient().describeConsumerGroups(Seq(group).asJava).describedGroups().get(group).get()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testListGroupApiWithAndWithoutListGroupAcls(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testListGroupApiWithAndWithoutListGroupAcls(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
// write some record to the topic
@ -1662,9 +1662,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
otherConsumer.close()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDeleteGroupApiWithDeleteGroupAcl(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testDeleteGroupApiWithDeleteGroupAcl(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource)
@ -1676,9 +1676,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
createAdminClient().deleteConsumerGroups(Seq(group).asJava).deletedGroups().get(group).get()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDeleteGroupApiWithNoDeleteGroupAcl(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testDeleteGroupApiWithNoDeleteGroupAcl(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource)
@ -1697,9 +1697,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
TestUtils.assertFutureExceptionTypeEquals(result.deletedGroups().get(group), classOf[GroupAuthorizationException])
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDeleteGroupOffsetsWithAcl(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testDeleteGroupOffsetsWithAcl(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, DELETE, ALLOW)), groupResource)
@ -1713,9 +1713,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
assertNull(result.partitionResult(tp).get())
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDeleteGroupOffsetsWithoutDeleteAcl(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testDeleteGroupOffsetsWithoutDeleteAcl(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource)
@ -1728,9 +1728,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
TestUtils.assertFutureExceptionTypeEquals(result.all(), classOf[GroupAuthorizationException])
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testDeleteGroupOffsetsWithDeleteAclWithoutTopicAcl(quorum: String, groupProtocol: String): Unit = {
createTopicWithBrokerPrincipal(topic)
// Create the consumer group
addAndVerifyAcls(Set(new AccessControlEntry(clientPrincipalString, WILDCARD_HOST, READ, ALLOW)), groupResource)
@ -2457,9 +2457,9 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
}
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testCreateAndCloseConsumerWithNoAccess(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
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.

View File

@ -25,9 +25,9 @@ import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, Deserializer, Serializer}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.MethodSource
import java.util.{Properties, stream}
import java.util.Properties
import java.util.concurrent.atomic.AtomicInteger
import scala.jdk.CollectionConverters._
import scala.collection.Seq
@ -112,31 +112,6 @@ abstract class BaseConsumerTest extends AbstractConsumerTest {
}
object BaseConsumerTest {
// We want to test the following combinations:
// * KRaft and the classic group protocol
// * KRaft and the consumer group protocol
def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = {
stream.Stream.of(
Arguments.of("kraft", "classic"),
Arguments.of("kraft", "consumer")
)
}
// For tests that only work with the classic group protocol, we want to test the following combinations:
// * KRaft and the classic group protocol
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly() : java.util.stream.Stream[Arguments] = {
stream.Stream.of(
Arguments.of("kraft", "classic")
)
}
// For tests that only work with the consumer group protocol, we want to test the following combination:
// * KRaft and the consumer group protocol
def getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly(): stream.Stream[Arguments] = {
stream.Stream.of(
Arguments.of("kraft", "consumer")
)
}
val updateProducerCount = new AtomicInteger()
val updateConsumerCount = new AtomicInteger()

View File

@ -24,7 +24,7 @@ import java.util.concurrent.TimeUnit
import kafka.integration.KafkaServerTestHarness
import kafka.security.JaasTestUtils
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.admin.{Admin, NewPartitions}
import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.clients.producer._
@ -38,7 +38,7 @@ import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import scala.collection.mutable
import scala.concurrent.ExecutionException
@ -81,7 +81,8 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
consumer = TestUtils.createConsumer(
bootstrapServers(listenerName = ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT)),
securityProtocol = SecurityProtocol.PLAINTEXT
groupProtocolFromTestParameters(),
securityProtocol = SecurityProtocol.PLAINTEXT,
)
}
@ -126,9 +127,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
* 1. Send with null key/value/partition-id should be accepted; send with null topic should be rejected.
* 2. Last message of the non-blocking send should return the correct offset metadata
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendOffset(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendOffset(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer()
val partition = 0
@ -188,9 +189,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendCompressedMessageWithCreateTime(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendCompressedMessageWithCreateTime(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(
compressionType = "gzip",
lingerMs = Int.MaxValue,
@ -198,9 +199,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendNonCompressedMessageWithCreateTime(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendNonCompressedMessageWithCreateTime(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue)
sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME)
}
@ -291,9 +292,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
*
* After close() returns, all messages should be sent with correct returned offset metadata
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testClose(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testClose(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer()
try {
@ -325,9 +326,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
*
* The specified partition-id should be respected
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendToPartition(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendToPartition(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer()
try {
@ -365,9 +366,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendToPartitionWithFollowerShutdownShouldNotTimeout(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_16176"))
def testSendToPartitionWithFollowerShutdownShouldNotTimeout(quorum: String, groupProtocol: String): Unit = {
// This test produces to a leader that has follower that is shutting down. It shows that
// the produce request succeed, do not timeout and do not need to be retried.
val producer = createProducer()
@ -418,9 +419,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
* Producer will attempt to send messages to the partition specified in each record, and should
* succeed as long as the partition is included in the metadata.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendBeforeAndAfterPartitionExpansion(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendBeforeAndAfterPartitionExpansion(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(maxBlockMs = 5 * 1000L)
// create topic
@ -476,9 +477,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
/**
* Test that flush immediately sends all accumulated requests.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFlush(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFlush(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue)
try {
TestUtils.createTopicWithAdmin(admin, topic, brokers, controllerServers, 2, 2)
@ -498,9 +499,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
/**
* Test close with zero timeout from caller thread
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testCloseWithZeroTimeoutFromCallerThread(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testCloseWithZeroTimeoutFromCallerThread(quorum: String, groupProtocol: String): Unit = {
TestUtils.createTopicWithAdmin(admin, topic, brokers, controllerServers, 2, 2)
val partition = 0
consumer.assign(List(new TopicPartition(topic, partition)).asJava)
@ -524,9 +525,9 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness {
/**
* Test close with zero and non-zero timeout from sender thread
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testCloseWithZeroTimeoutFromSenderThread(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testCloseWithZeroTimeoutFromSenderThread(quorum: String, groupProtocol: String): Unit = {
TestUtils.createTopicWithAdmin(admin, topic, brokers, controllerServers, 1, 2)
val partition = 0
consumer.assign(List(new TopicPartition(topic, partition)).asJava)

View File

@ -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.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(QuotaConfig.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)

View File

@ -17,7 +17,7 @@ import java.{time, util}
import java.util.concurrent._
import java.util.{Collections, Properties}
import kafka.server.KafkaConfig
import kafka.utils.{Logging, TestUtils}
import kafka.utils.{Logging, TestInfoUtils, TestUtils}
import org.apache.kafka.clients.consumer._
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition
@ -30,6 +30,8 @@ import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.util.ShutdownableThread
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, Disabled, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import java.time.Duration
import scala.jdk.CollectionConverters._
@ -78,8 +80,9 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
}
}
@Test
def testConsumptionWithBrokerFailures(): Unit = consumeWithBrokerFailures(10)
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testConsumptionWithBrokerFailures(quorum: String, groupProtocol: String): Unit = consumeWithBrokerFailures(10)
/*
* 1. Produce a bunch of messages
@ -122,8 +125,9 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
}
}
@Test
def testSeekAndCommitWithBrokerFailures(): Unit = seekAndCommitWithBrokerFailures(5)
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testSeekAndCommitWithBrokerFailures(quorum: String, groupProtocol: String): Unit = seekAndCommitWithBrokerFailures(5)
def seekAndCommitWithBrokerFailures(numIters: Int): Unit = {
val numRecords = 1000
@ -165,8 +169,9 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
}
}
@Test
def testSubscribeWhenTopicUnavailable(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testSubscribeWhenTopicUnavailable(quorum: String, groupProtocol: String): Unit = {
val numRecords = 1000
val newtopic = "newtopic"
@ -216,8 +221,9 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
receiveExactRecords(poller, numRecords, 10000L)
}
@Test
def testClose(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testClose(quorum: String, groupProtocol: String): Unit = {
val numRecords = 10
val producer = createProducer()
producerSend(producer, numRecords)
@ -341,8 +347,9 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
/**
* When we have the consumer group max size configured to X, the X+1th consumer trying to join should receive a fatal exception
*/
@Test
def testConsumerReceivesFatalExceptionWhenGroupPassesMaxSize(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testConsumerReceivesFatalExceptionWhenGroupPassesMaxSize(quorum: String, groupProtocol: String): Unit = {
val group = "fatal-exception-test"
val topic = "fatal-exception-test"
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "60000")
@ -379,8 +386,9 @@ class ConsumerBounceTest extends AbstractConsumerTest with Logging {
* immediately if rebalance is in progress. If brokers are not available,
* close should terminate immediately without sending leave group.
*/
@Test
def testCloseDuringRebalance(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testCloseDuringRebalance(quorum: String, groupProtocol: String): Unit = {
val topic = "closetest"
createTopic(topic, 10, brokerCount)
this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, "60000")

View File

@ -16,15 +16,17 @@
*/
package kafka.api
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import java.util.Collections
class ConsumerRebootstrapTest extends RebootstrapTest {
@Test
def testRebootstrap(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testRebootstrap(quorum: String, groupProtocol: String): Unit = {
sendRecords(10, 0)
TestUtils.waitUntilTrue(

View File

@ -20,10 +20,10 @@ package kafka.api
import java.lang.{Boolean => JBoolean}
import java.time.Duration
import java.util
import java.util.Collections
import java.util.{Collections, Locale}
import kafka.utils.{EmptyTestInfo, TestUtils}
import org.apache.kafka.clients.admin.NewTopic
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.clients.consumer.{ConsumerConfig, GroupProtocol}
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
import org.apache.kafka.server.config.{ServerConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
@ -35,10 +35,10 @@ import org.junit.jupiter.params.provider.{Arguments, MethodSource}
*/
class ConsumerTopicCreationTest {
@ParameterizedTest
@ParameterizedTest(name = "{displayName}.groupProtocol={0}.brokerAutoTopicCreationEnable={1}.consumerAllowAutoCreateTopics={2}")
@MethodSource(Array("parameters"))
def testAutoTopicCreation(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean): Unit = {
val testCase = new ConsumerTopicCreationTest.TestCase(brokerAutoTopicCreationEnable, consumerAllowAutoCreateTopics)
def testAutoTopicCreation(groupProtocol: String, brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean): Unit = {
val testCase = new ConsumerTopicCreationTest.TestCase(groupProtocol, brokerAutoTopicCreationEnable, consumerAllowAutoCreateTopics)
testCase.setUp(new EmptyTestInfo())
try testCase.test() finally testCase.tearDown()
}
@ -47,7 +47,7 @@ class ConsumerTopicCreationTest {
object ConsumerTopicCreationTest {
private class TestCase(brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean) extends IntegrationTestHarness {
private class TestCase(groupProtocol: String, brokerAutoTopicCreationEnable: JBoolean, consumerAllowAutoCreateTopics: JBoolean) extends IntegrationTestHarness {
private val topic_1 = "topic-1"
private val topic_2 = "topic-2"
private val producerClientId = "ConsumerTestProducer"
@ -65,6 +65,7 @@ object ConsumerTopicCreationTest {
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false")
this.consumerConfig.setProperty(ConsumerConfig.METADATA_MAX_AGE_CONFIG, "100")
this.consumerConfig.setProperty(ConsumerConfig.ALLOW_AUTO_CREATE_TOPICS_CONFIG, consumerAllowAutoCreateTopics.toString)
this.consumerConfig.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
override protected def brokerCount: Int = 1
@ -99,7 +100,7 @@ object ConsumerTopicCreationTest {
val data = new java.util.ArrayList[Arguments]()
for (brokerAutoTopicCreationEnable <- Array(JBoolean.TRUE, JBoolean.FALSE))
for (consumerAutoCreateTopicsPolicy <- Array(JBoolean.TRUE, JBoolean.FALSE))
data.add(Arguments.of(brokerAutoTopicCreationEnable, consumerAutoCreateTopicsPolicy))
data.add(Arguments.of(GroupProtocol.CLASSIC.name.toLowerCase(Locale.ROOT), brokerAutoTopicCreationEnable, consumerAutoCreateTopicsPolicy))
data.stream()
}
}

View File

@ -16,13 +16,14 @@
*/
package kafka.api
import kafka.utils.TestInfoUtils
import org.apache.kafka.clients.producer.ProducerConfig
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.server.config.ReplicationConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertThrows}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import java.util
import java.util.{Collections, Optional, Properties}
@ -39,9 +40,9 @@ class ConsumerWithLegacyMessageFormatIntegrationTest extends AbstractConsumerTes
}
@nowarn("cat=deprecation")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testOffsetsForTimes(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testOffsetsForTimes(quorum: String, groupProtocol: String): Unit = {
val numParts = 2
val topic1 = "part-test-topic-1"
val topic2 = "part-test-topic-2"
@ -116,9 +117,9 @@ class ConsumerWithLegacyMessageFormatIntegrationTest extends AbstractConsumerTes
}
@nowarn("cat=deprecation")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testEarliestOrLatestOffsets(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testEarliestOrLatestOffsets(quorum: String, groupProtocol: String): Unit = {
val topic0 = "topicWithNewMessageFormat"
val topic1 = "topicWithOldMessageFormat"
val prop = new Properties()

View File

@ -18,7 +18,7 @@ import kafka.api.GroupedUserPrincipalBuilder._
import kafka.api.GroupedUserQuotaCallback._
import kafka.security.{JaasModule, JaasTestUtils}
import kafka.server._
import kafka.utils.{Logging, TestUtils}
import kafka.utils.{Logging, TestInfoUtils, TestUtils}
import kafka.zk.ConfigEntityChangeNotificationZNode
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig}
@ -31,7 +31,9 @@ import org.apache.kafka.common.{Cluster, Reconfigurable}
import org.apache.kafka.server.config.{QuotaConfig, ServerConfigs}
import org.apache.kafka.server.quota._
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import java.util.Properties
import java.util.concurrent.ConcurrentHashMap
@ -88,8 +90,9 @@ class CustomQuotaCallbackTest extends IntegrationTestHarness with SaslSetup {
createScramCredentials(zkConnect, JaasTestUtils.KAFKA_SCRAM_ADMIN, JaasTestUtils.KAFKA_SCRAM_ADMIN_PASSWORD)
}
@Test
def testCustomQuotaCallback(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testCustomQuotaCallback(quorum: String, groupProtocol: String): Unit = {
// Large quota override, should not throttle
var brokerId = 0
var user = createGroupWithOneUser("group0_user1", brokerId)

View File

@ -28,7 +28,7 @@ import org.apache.kafka.common.security.token.delegation.DelegationToken
import org.apache.kafka.metadata.storage.Formatter
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import org.junit.jupiter.api.{BeforeEach, TestInfo}
import scala.jdk.CollectionConverters._
@ -99,9 +99,9 @@ class DelegationTokenEndToEndAuthorizationTest extends EndToEndAuthorizationTest
superuserClientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, privilegedClientLoginContext)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testCreateUserWithDelegationToken(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testCreateUserWithDelegationToken(quorum: String, groupProtocol: String): Unit = {
val privilegedAdminClient = Admin.create(privilegedAdminClientConfig)
try {
val user = "user"

View File

@ -38,12 +38,12 @@ 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}
import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, 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.{CsvSource, ValueSource}
import org.junit.jupiter.params.provider.{CsvSource, MethodSource}
import scala.jdk.CollectionConverters._
@ -170,9 +170,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"))
def testProduceConsumeViaAssign(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testProduceConsumeViaAssign(quorum: String, groupProtocol: String): Unit = {
setAclsAndProduce(tp)
val consumer = createConsumer()
consumer.assign(List(tp).asJava)
@ -199,9 +199,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
._2.asInstanceOf[Gauge[Double]]
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testProduceConsumeViaSubscribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testProduceConsumeViaSubscribe(quorum: String, groupProtocol: String): Unit = {
setAclsAndProduce(tp)
val consumer = createConsumer()
consumer.subscribe(List(topic).asJava)
@ -209,9 +209,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
confirmReauthenticationMetrics()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testProduceConsumeWithWildcardAcls(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testProduceConsumeWithWildcardAcls(quorum: String, groupProtocol: String): Unit = {
setWildcardResourceAcls()
val producer = createProducer()
sendRecords(producer, numRecords, tp)
@ -221,9 +221,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
confirmReauthenticationMetrics()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testProduceConsumeWithPrefixedAcls(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testProduceConsumeWithPrefixedAcls(quorum: String, groupProtocol: String): Unit = {
setPrefixedResourceAcls()
val producer = createProducer()
sendRecords(producer, numRecords, tp)
@ -233,9 +233,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
confirmReauthenticationMetrics()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testProduceConsumeTopicAutoCreateTopicCreateAcl(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testProduceConsumeTopicAutoCreateTopicCreateAcl(quorum: String, groupProtocol: String): Unit = {
// topic2 is not created on setup()
val tp2 = new TopicPartition("topic2", 0)
setAclsAndProduce(tp2)
@ -301,12 +301,14 @@ 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 = "{displayName}.quorum={0}.groupProtocol={1}.isIdempotenceEnabled={2}")
@CsvSource(value = Array(
"kraft, true",
"kraft, false",
"kraft, classic, true",
//"kraft, consumer, true",
"kraft, classic, false",
//"kraft, consumer, 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()
@ -401,9 +403,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"))
def testNoConsumeWithoutDescribeAclViaAssign(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testNoConsumeWithoutDescribeAclViaAssign(quorum: String, groupProtocol: String): Unit = {
noConsumeWithoutDescribeAclSetup()
val consumer = createConsumer()
consumer.assign(List(tp).asJava)
@ -412,9 +414,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
confirmReauthenticationMetrics()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testNoConsumeWithoutDescribeAclViaSubscribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testNoConsumeWithoutDescribeAclViaSubscribe(quorum: String, groupProtocol: String): Unit = {
noConsumeWithoutDescribeAclSetup()
val consumer = createConsumer()
consumer.subscribe(List(topic).asJava)
@ -453,9 +455,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
}
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testNoConsumeWithDescribeAclViaAssign(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testNoConsumeWithDescribeAclViaAssign(quorum: String, groupProtocol: String): Unit = {
noConsumeWithDescribeAclSetup()
val consumer = createConsumer()
consumer.assign(List(tp).asJava)
@ -465,9 +467,9 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas
confirmReauthenticationMetrics()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testNoConsumeWithDescribeAclViaSubscribe(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testNoConsumeWithDescribeAclViaSubscribe(quorum: String, groupProtocol: String): Unit = {
noConsumeWithDescribeAclSetup()
val consumer = createConsumer()
consumer.subscribe(List(topic).asJava)
@ -494,9 +496,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"))
def testNoGroupAcl(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testNoGroupAcl(quorum: String, groupProtocol: String): Unit = {
val superuserAdminClient = createSuperuserAdminClient()
superuserAdminClient.createAcls(List(AclTopicWrite(), AclTopicCreate(), AclTopicDescribe()).asJava).values
brokers.foreach { s =>

View File

@ -35,7 +35,7 @@ import org.junit.jupiter.api.{BeforeEach, TestInfo}
import scala.jdk.CollectionConverters._
import org.apache.kafka.test.TestUtils.isValidClusterId
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
/** The test cases here verify the following conditions.
* 1. The ProducerInterceptor receives the cluster id after the onSend() method is called and before onAcknowledgement() method is called.
@ -116,9 +116,9 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness {
createTopic(topic, 2, serverCount)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testEndToEnd(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testEndToEnd(quorum: String, groupProtocol: String): Unit = {
val appendStr = "mock"
MockConsumerInterceptor.resetCounters()
MockProducerInterceptor.resetCounters()
@ -151,6 +151,7 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness {
this.consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
this.consumerConfig.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, classOf[MockConsumerInterceptor].getName)
this.consumerConfig.put(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, classOf[MockConsumerMetricsReporter].getName)
this.consumerConfig.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
val testConsumer = new KafkaConsumer(this.consumerConfig, new MockDeserializer, new MockDeserializer)
testConsumer.assign(List(tp).asJava)
testConsumer.seek(tp, 0)

View File

@ -16,7 +16,7 @@ import java.util.Properties
import java.util.concurrent.ExecutionException
import kafka.api.GroupAuthorizerIntegrationTest._
import kafka.server.BaseRequestTest
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.TopicPartition
@ -35,7 +35,7 @@ import org.apache.kafka.server.config.ServerConfigs
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.MethodSource
import scala.jdk.CollectionConverters._
@ -111,9 +111,9 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest {
new AccessControlEntry(principal.toString, WILDCARD_HOST, aclOperation, aclPermissionType)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testUnauthorizedProduceAndConsume(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testUnauthorizedProduceAndConsume(quorum: String, groupProtocol: String): Unit = {
val topic = "topic"
val topicPartition = new TopicPartition("topic", 0)
@ -132,9 +132,9 @@ class GroupAuthorizerIntegrationTest extends BaseRequestTest {
assertEquals(Set(topic), consumeException.unauthorizedTopics.asScala)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testAuthorizedProduceAndConsume(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testAuthorizedProduceAndConsume(quorum: String, groupProtocol: String): Unit = {
val topic = "topic"
val topicPartition = new TopicPartition("topic", 0)

View File

@ -304,8 +304,8 @@ class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) {
)(f: Consumer[Array[Byte], Array[Byte]] => Unit): Unit = {
val consumer = TestUtils.createConsumer(
brokerList = cluster.bootstrapServers(),
groupId = groupId,
groupProtocol = groupProtocol,
groupId = groupId,
enableAutoCommit = enableAutoCommit
)
try {

View File

@ -155,7 +155,7 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
consumerConfig.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, "group")
consumerConfig.putIfAbsent(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName)
consumerConfig.putIfAbsent(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, classOf[ByteArrayDeserializer].getName)
maybeGroupProtocolSpecified(testInfo).map(groupProtocol => consumerConfig.putIfAbsent(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name))
maybeGroupProtocolSpecified().map(groupProtocol => consumerConfig.putIfAbsent(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.name))
shareConsumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
shareConsumerConfig.putIfAbsent(ConsumerConfig.GROUP_ID_CONFIG, "group")
@ -195,6 +195,9 @@ abstract class IntegrationTestHarness extends KafkaServerTestHarness {
valueDeserializer: Deserializer[V] = new ByteArrayDeserializer,
configOverrides: Properties = new Properties,
configsToRemove: List[String] = List()): Consumer[K, V] = {
if (!consumerConfig.containsKey(ConsumerConfig.GROUP_PROTOCOL_CONFIG))
throw new IllegalStateException(s"Please specify the group.protocol configuration when creating a KafkaConsumer")
val props = new Properties
props ++= consumerConfig
props ++= configOverrides

View File

@ -18,7 +18,7 @@ package kafka.api
import java.util.Collections
import java.util.concurrent.TimeUnit
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.record.TimestampType
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
@ -26,7 +26,7 @@ import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.{BeforeEach, TestInfo}
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertTrue}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
/**
* Tests where the broker is configured to use LogAppendTime. For tests where LogAppendTime is configured via topic
@ -49,9 +49,9 @@ class LogAppendTimeTest extends IntegrationTestHarness {
createTopic(topic)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testProduceConsume(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testProduceConsume(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer()
val now = System.currentTimeMillis()
val createTime = now - TimeUnit.DAYS.toMillis(1)
@ -77,5 +77,4 @@ class LogAppendTimeTest extends IntegrationTestHarness {
assertEquals(TimestampType.LOG_APPEND_TIME, consumerRecord.timestampType)
}
}
}

View File

@ -30,7 +30,7 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.CsvSource
import java.util.{Locale, Properties}
import scala.annotation.nowarn
@ -81,9 +81,9 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {
* Verifies some of the metrics of producer, consumer as well as server.
*/
@nowarn("cat=deprecation")
@ParameterizedTest(name = "testMetrics with systemRemoteStorageEnabled: {0}")
@ValueSource(booleans = Array(true, false))
def testMetrics(systemRemoteStorageEnabled: Boolean): Unit = {
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}.systemRemoteStorageEnabled={2}")
@CsvSource(Array("zk,classic,true", "zk,classic,false"))
def testMetrics(quorum: String, groupProtocol: String, systemRemoteStorageEnabled: Boolean): Unit = {
val topic = "topicWithOldMessageFormat"
val props = new Properties
props.setProperty(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.9.0")

View File

@ -33,8 +33,7 @@ import kafka.utils.{Log4jController, TestInfoUtils, TestUtils}
import org.apache.kafka.clients.HostResolver
import org.apache.kafka.clients.admin.ConfigEntry.ConfigSource
import org.apache.kafka.clients.admin._
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, KafkaConsumer, OffsetAndMetadata, ShareConsumer}
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, GroupProtocol, KafkaConsumer, OffsetAndMetadata, ShareConsumer}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
import org.apache.kafka.common.acl.{AccessControlEntry, AclBinding, AclBindingFilter, AclOperation, AclPermissionType}
import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, SslConfigs, TopicConfig}
@ -57,7 +56,7 @@ import org.apache.log4j.PropertyConfigurator
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, MethodSource, ValueSource}
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
import org.slf4j.LoggerFactory
import java.util.AbstractMap.SimpleImmutableEntry
@ -235,10 +234,10 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
Admin.create(config)
}
@ParameterizedTest
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
@Timeout(10)
@ValueSource(strings = Array("kraft"))
def testDescribeUserScramCredentialsTimeout(quorum: String): Unit = {
def testDescribeUserScramCredentialsTimeout(quorum: String, groupProtocol: String): Unit = {
client = createInvalidAdminClient()
try {
// test describeUserScramCredentials(List<String> users, DescribeUserScramCredentialsOptions options)
@ -250,10 +249,11 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
} finally client.close(time.Duration.ZERO)
}
private def consumeToExpectedNumber = (expectedNumber: Int) => {
private def consumeToExpectedNumber = (expectedNumber: Int, groupProtocol: String) => {
val configs = new util.HashMap[String, Object]()
configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, plaintextBootstrapServers(brokers))
configs.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString)
configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
val consumer = new KafkaConsumer(configs, new ByteArrayDeserializer, new ByteArrayDeserializer)
try {
consumer.assign(Collections.singleton(topicPartition))
@ -267,9 +267,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
} finally consumer.close()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDescribeProducers(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testDescribeProducers(quorum: String, groupProtocol: String): Unit = {
client = createAdminClient
client.createTopics(Collections.singletonList(new NewTopic(topic, 1, 1.toShort))).all().get()
@ -316,7 +316,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
// send committed transaction msg
appendTransactionRecords("foo", 2, commit = true)
// consume 3 records to ensure transaction finished
consumeToExpectedNumber(3)
consumeToExpectedNumber(3, groupProtocol)
val transactionProducerIterator = queryProducerDetail()
assertEquals(2, transactionProducerIterator.size)
val containsCoordinatorEpochIterator = transactionProducerIterator
@ -529,9 +529,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
} finally producerNew.close()
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testAbortTransaction(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testAbortTransaction(quorum: String, groupProtocol: String): Unit = {
client = createAdminClient
val tp = new TopicPartition("topic1", 0)
client.createTopics(Collections.singletonList(new NewTopic(tp.topic(), 1, 1.toShort))).all().get()
@ -540,8 +540,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
val configs = new util.HashMap[String, Object]()
configs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, plaintextBootstrapServers(brokers))
configs.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, IsolationLevel.READ_COMMITTED.toString)
if (quorum == "kraft")
configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, ConsumerProtocol.PROTOCOL_TYPE)
configs.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
val consumer = new KafkaConsumer(configs, new ByteArrayDeserializer, new ByteArrayDeserializer)
try {
consumer.assign(Collections.singleton(tp))
@ -856,9 +855,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testAlterReplicaLogDirs(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testAlterReplicaLogDirs(quorum: String, groupProtocol: String): Unit = {
client = createAdminClient
val topic = "topic"
val tp = new TopicPartition(topic, 0)
@ -940,7 +939,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
// Verify that all messages that are produced can be consumed
val consumerRecords = TestUtils.consumeTopicRecords(brokers, topic, finalNumMessages,
securityProtocol = securityProtocol, trustStoreFile = trustStoreFile)
GroupProtocol.of(groupProtocol), securityProtocol = securityProtocol, trustStoreFile = trustStoreFile)
consumerRecords.zipWithIndex.foreach { case (consumerRecord, index) =>
assertEquals(s"xxxxxxxxxxxxxxxxxxxx-$index", new String(consumerRecord.value))
}
@ -1341,9 +1340,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertEquals("This server does not host this topic-partition.", e.getCause.getMessage)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testSeekAfterDeleteRecords(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSeekAfterDeleteRecords(quorum: String, groupProtocol: String): Unit = {
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
client = createAdminClient
@ -1371,9 +1370,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertEquals(10L, consumer.position(topicPartition))
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testLogStartOffsetCheckpoint(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testLogStartOffsetCheckpoint(quorum: String, groupProtocol: String): Unit = {
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
client = createAdminClient
@ -1411,9 +1410,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}, s"Expected low watermark of the partition to be 5 but got ${lowWatermark.getOrElse("no response within the timeout")}")
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testLogStartOffsetAfterDeleteRecords(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testLogStartOffsetAfterDeleteRecords(quorum: String, groupProtocol: String): Unit = {
createTopic(topic, numPartitions = 2, replicationFactor = brokerCount)
client = createAdminClient
@ -1544,9 +1543,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
assertNull(returnedOffsets.get(topicPartition))
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testConsumeAfterDeleteRecords(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testConsumeAfterDeleteRecords(quorum: String, groupProtocol: String): Unit = {
val consumer = createConsumer()
subscribeAndWaitForAssignment(topic, consumer)
@ -1568,9 +1567,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
TestUtils.consumeRecords(consumer, 2)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDeleteRecordsWithException(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testDeleteRecordsWithException(quorum: String, groupProtocol: String): Unit = {
val consumer = createConsumer()
subscribeAndWaitForAssignment(topic, consumer)
@ -1747,9 +1746,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
/**
* Test the consumer group APIs.
*/
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testConsumerGroups(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17960"))
def testConsumerGroups(quorum: String, groupProtocol: String): Unit = {
val config = createConfig
client = Admin.create(config)
try {
@ -1824,6 +1823,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
// Start consumers in a thread that will subscribe to a new group.
val consumerThreads = consumerSet.zip(topicSet).map(zipped => createConsumerThread(zipped._1, zipped._2))
val groupType = if (groupProtocol.equalsIgnoreCase(GroupProtocol.CONSUMER.name)) GroupType.CONSUMER else GroupType.CLASSIC
try {
consumerThreads.foreach(_.start())
@ -1837,21 +1837,21 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}, s"Expected to be able to list $testGroupId")
TestUtils.waitUntilTrue(() => {
val options = new ListConsumerGroupsOptions().withTypes(Set(GroupType.CLASSIC).asJava)
val options = new ListConsumerGroupsOptions().withTypes(Set(groupType).asJava)
val matching = client.listConsumerGroups(options).all.get.asScala.filter(group =>
group.groupId == testGroupId &&
group.state.get == ConsumerGroupState.STABLE)
matching.size == 1
}, s"Expected to be able to list $testGroupId in group type Classic")
}, s"Expected to be able to list $testGroupId in group type $groupType")
TestUtils.waitUntilTrue(() => {
val options = new ListConsumerGroupsOptions().withTypes(Set(GroupType.CLASSIC).asJava)
val options = new ListConsumerGroupsOptions().withTypes(Set(groupType).asJava)
.inStates(Set(ConsumerGroupState.STABLE).asJava)
val matching = client.listConsumerGroups(options).all.get.asScala.filter(group =>
group.groupId == testGroupId &&
group.state.get == ConsumerGroupState.STABLE)
matching.size == 1
}, s"Expected to be able to list $testGroupId in group type Classic and state Stable")
}, s"Expected to be able to list $testGroupId in group type $groupType and state Stable")
TestUtils.waitUntilTrue(() => {
val options = new ListConsumerGroupsOptions().inStates(Set(ConsumerGroupState.STABLE).asJava)
@ -2016,9 +2016,9 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
}
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testDeleteConsumerGroupOffsets(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testDeleteConsumerGroupOffsets(quorum: String, groupProtocol: String): Unit = {
val config = createConfig
client = Admin.create(config)
try {
@ -3569,13 +3569,6 @@ object PlaintextAdminIntegrationTest {
assertEquals(LogConfig.DEFAULT_COMPRESSION_TYPE, configs.get(brokerResource).get(ServerConfigs.COMPRESSION_TYPE_CONFIG).value)
}
def getTestQuorumAndGroupProtocolParametersAll() : java.util.stream.Stream[Arguments] = {
util.Arrays.stream(Array(
Arguments.of("kraft", "classic"),
Arguments.of("kraft", "consumer")
))
}
/**
* Resets the logging configuration after the test.
*/

View File

@ -19,9 +19,8 @@ import org.apache.kafka.common.TopicPartition
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.MethodSource
import java.util.stream.Stream
import scala.jdk.CollectionConverters._
/**
@ -206,8 +205,3 @@ class PlaintextConsumerAssignTest extends AbstractConsumerTest {
}
}
object PlaintextConsumerAssignTest {
def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll()
}

View File

@ -19,12 +19,11 @@ import org.apache.kafka.common.errors.UnsupportedAssignorException
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource}
import org.junit.jupiter.params.provider.{CsvSource, MethodSource}
import java.util
import java.util.concurrent.TimeUnit
import java.util.concurrent.locks.ReentrantLock
import java.util.stream.Stream
import scala.collection.mutable
import scala.jdk.CollectionConverters._
@ -307,16 +306,16 @@ class PlaintextConsumerAssignorsTest extends AbstractConsumerTest {
}
// Only the classic group protocol supports client-side assignors
@ParameterizedTest
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}.assignmentStrategy={2}")
@CsvSource(Array(
"org.apache.kafka.clients.consumer.CooperativeStickyAssignor, zk",
"org.apache.kafka.clients.consumer.RangeAssignor, zk",
"org.apache.kafka.clients.consumer.CooperativeStickyAssignor, kraft",
"org.apache.kafka.clients.consumer.RangeAssignor, kraft"
"zk, classic, org.apache.kafka.clients.consumer.CooperativeStickyAssignor",
"zk, classic, org.apache.kafka.clients.consumer.RangeAssignor",
"kraft, classic, org.apache.kafka.clients.consumer.CooperativeStickyAssignor",
"kraft, classic, org.apache.kafka.clients.consumer.RangeAssignor"
))
def testRebalanceAndRejoin(assignmentStrategy: String, quorum: String): Unit = {
def testRebalanceAndRejoin(quorum: String, groupProtocol: String, assignmentStrategy: String): Unit = {
// create 2 consumers
this.consumerConfig.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, "classic")
this.consumerConfig.setProperty(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "rebalance-and-rejoin-group")
this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, assignmentStrategy)
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
@ -396,8 +395,3 @@ class PlaintextConsumerAssignorsTest extends AbstractConsumerTest {
}
}
object PlaintextConsumerAssignorsTest {
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly()
}

View File

@ -12,19 +12,18 @@
*/
package integration.kafka.api
import kafka.api.{AbstractConsumerTest, BaseConsumerTest}
import kafka.api.AbstractConsumerTest
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.consumer.{Consumer, ConsumerRebalanceListener}
import org.apache.kafka.common.TopicPartition
import org.junit.jupiter.api.Assertions.{assertDoesNotThrow, assertEquals, assertThrows, assertTrue}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.MethodSource
import java.util
import java.util.Arrays.asList
import java.util.Collections
import java.util.concurrent.atomic.AtomicBoolean
import java.util.stream.Stream
/**
* Integration tests for the consumer that cover interaction with the consumer from within callbacks
@ -161,11 +160,3 @@ class PlaintextConsumerCallbackTest extends AbstractConsumerTest {
assertTrue(partitionsRevoked.get())
}
}
object PlaintextConsumerCallbackTest {
def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll()
}

View File

@ -21,12 +21,11 @@ import org.apache.kafka.test.MockConsumerInterceptor
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.MethodSource
import java.time.Duration
import java.util
import java.util.Optional
import java.util.stream.Stream
import scala.jdk.CollectionConverters._
/**
@ -370,11 +369,3 @@ class PlaintextConsumerCommitTest extends AbstractConsumerTest {
awaitAssignment(consumer, expectedAssignment)
}
}
object PlaintextConsumerCommitTest {
def getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly()
def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll()
}

View File

@ -18,11 +18,10 @@ import org.apache.kafka.clients.producer.ProducerRecord
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.MethodSource
import org.apache.kafka.common.TopicPartition
import java.time.Duration
import java.util.stream.Stream
import scala.jdk.CollectionConverters._
/**
@ -236,8 +235,3 @@ class PlaintextConsumerFetchTest extends AbstractConsumerTest {
}
}
object PlaintextConsumerFetchTest {
def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll()
}

View File

@ -19,11 +19,10 @@ import org.apache.kafka.common.utils.Utils
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.MethodSource
import java.time.Duration
import java.util
import java.util.stream.Stream
import scala.collection.mutable
import scala.jdk.CollectionConverters._
@ -290,11 +289,3 @@ class PlaintextConsumerPollTest extends AbstractConsumerTest {
poller.shutdown()
}
}
object PlaintextConsumerPollTest {
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly()
def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll()
}

View File

@ -20,11 +20,10 @@ import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.api.function.Executable
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import org.junit.jupiter.params.provider.MethodSource
import java.time.Duration
import java.util.regex.Pattern
import java.util.stream.Stream
import scala.jdk.CollectionConverters._
/**
@ -270,8 +269,3 @@ class PlaintextConsumerSubscriptionTest extends AbstractConsumerTest {
assertEquals(s"Invalid topics: [${invalidTopicName}]", exception.getMessage)
}
}
object PlaintextConsumerSubscriptionTest {
def getTestQuorumAndGroupProtocolParametersAll: Stream[Arguments] =
BaseConsumerTest.getTestQuorumAndGroupProtocolParametersAll()
}

View File

@ -17,9 +17,10 @@
package kafka.api
import java.util.Properties
import java.util.{Locale, Properties}
import java.util.concurrent.{ExecutionException, Future, TimeUnit}
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.consumer.GroupProtocol
import org.apache.kafka.clients.producer.{BufferExhaustedException, KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata}
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors.{InvalidTimestampException, RecordTooLargeException, SerializationException, TimeoutException}
@ -29,7 +30,7 @@ import org.apache.kafka.storage.internals.log.LogConfig
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{Arguments, MethodSource, ValueSource}
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
import java.nio.charset.StandardCharsets
import scala.annotation.nowarn
@ -37,9 +38,9 @@ import scala.annotation.nowarn
class PlaintextProducerSendTest extends BaseProducerSendTest {
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testWrongSerializer(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testWrongSerializer(quorum: String, groupProtocol: String): Unit = {
val producerProps = new Properties()
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer")
@ -49,9 +50,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
assertThrows(classOf[SerializationException], () => producer.send(record))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testBatchSizeZero(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testBatchSizeZero(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(
lingerMs = Int.MaxValue,
deliveryTimeoutMs = Int.MaxValue,
@ -60,9 +61,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
}
@Timeout(value = 15, unit = TimeUnit.SECONDS, threadMode = Timeout.ThreadMode.SEPARATE_THREAD)
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testBatchSizeZeroNoPartitionNoRecordKey(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testBatchSizeZeroNoPartitionNoRecordKey(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(batchSize = 0)
val numRecords = 10
try {
@ -83,9 +84,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendCompressedMessageWithLogAppendTime(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendCompressedMessageWithLogAppendTime(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(
compressionType = "gzip",
lingerMs = Int.MaxValue,
@ -93,9 +94,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendNonCompressedMessageWithLogAppendTime(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendNonCompressedMessageWithLogAppendTime(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer(lingerMs = Int.MaxValue, deliveryTimeoutMs = Int.MaxValue)
sendAndVerifyTimestamp(producer, TimestampType.LOG_APPEND_TIME)
}
@ -105,9 +106,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
*
* The topic should be created upon sending the first message
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testAutoCreateTopic(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testAutoCreateTopic(quorum: String, groupProtocol: String): Unit = {
val producer = createProducer()
try {
// Send a message to auto-create the topic
@ -121,9 +122,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
}
}
@ParameterizedTest
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("quorumAndTimestampConfigProvider"))
def testSendWithInvalidBeforeAndAfterTimestamp(quorum: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
def testSendWithInvalidBeforeAndAfterTimestamp(quorum: String, groupProtocol: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
val topicProps = new Properties()
// set the TopicConfig for timestamp validation to have 1 minute threshold. Note that recordTimestamp has 5 minutes diff
val oneMinuteInMs: Long = 1 * 60 * 60 * 1000L
@ -150,9 +151,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
}
}
@ParameterizedTest
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("quorumAndTimestampConfigProvider"))
def testValidBeforeAndAfterTimestampsAtThreshold(quorum: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
def testValidBeforeAndAfterTimestampsAtThreshold(quorum: String, groupProtocol: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
val topicProps = new Properties()
// set the TopicConfig for timestamp validation to be the same as the record timestamp
@ -170,9 +171,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
compressedProducer.close()
}
@ParameterizedTest
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("quorumAndTimestampConfigProvider"))
def testValidBeforeAndAfterTimestampsWithinThreshold(quorum: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
def testValidBeforeAndAfterTimestampsWithinThreshold(quorum: String, groupProtocol: String, messageTimeStampConfig: String, recordTimestamp: Long): Unit = {
val topicProps = new Properties()
// set the TopicConfig for timestamp validation to have 10 minute threshold. Note that recordTimestamp has 5 minutes diff
@ -194,9 +195,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
// Test that producer with max.block.ms=0 can be used to send in non-blocking mode
// where requests are failed immediately without blocking if metadata is not available
// or buffer is full.
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testNonBlockingProducer(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testNonBlockingProducer(quorum: String, groupProtocol: String): Unit = {
def send(producer: KafkaProducer[Array[Byte],Array[Byte]]): Future[RecordMetadata] = {
producer.send(new ProducerRecord(topic, 0, "key".getBytes, new Array[Byte](1000)))
@ -250,9 +251,9 @@ class PlaintextProducerSendTest extends BaseProducerSendTest {
verifySendSuccess(future2) // previous batch should be completed and sent now
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendRecordBatchWithMaxRequestSizeAndHigher(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendRecordBatchWithMaxRequestSizeAndHigher(quorum: String, groupProtocol: String): Unit = {
val producerProps = new Properties()
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
val producer = registerProducer(new KafkaProducer(producerProps, new ByteArraySerializer, new ByteArraySerializer))
@ -280,13 +281,12 @@ object PlaintextProducerSendTest {
def quorumAndTimestampConfigProvider: java.util.stream.Stream[Arguments] = {
val now: Long = System.currentTimeMillis()
val fiveMinutesInMs: Long = 5 * 60 * 60 * 1000L
java.util.stream.Stream.of[Arguments](
Arguments.of("zk", TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
Arguments.of("zk", TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
Arguments.of("zk", TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.box(now + fiveMinutesInMs)),
Arguments.of("kraft", TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
Arguments.of("kraft", TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)),
Arguments.of("kraft", TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.box(now + fiveMinutesInMs))
)
val data = new java.util.ArrayList[Arguments]()
for (groupProtocol <- GroupProtocol.values().map(gp => gp.name.toLowerCase(Locale.ROOT))) {
data.add(Arguments.of("kraft", groupProtocol, TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)))
data.add(Arguments.of("kraft", groupProtocol, TopicConfig.MESSAGE_TIMESTAMP_BEFORE_MAX_MS_CONFIG, Long.box(now - fiveMinutesInMs)))
data.add(Arguments.of("kraft", groupProtocol, TopicConfig.MESSAGE_TIMESTAMP_AFTER_MAX_MS_CONFIG, Long.box(now + fiveMinutesInMs)))
}
data.stream()
}
}

View File

@ -19,6 +19,7 @@ package kafka.api.test
import kafka.server.{KafkaBroker, KafkaConfig, QuorumTestHarness}
import kafka.utils.TestUtils
import org.apache.kafka.clients.consumer.GroupProtocol
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.header.Header
@ -62,17 +63,20 @@ class ProducerCompressionTest extends QuorumTestHarness {
*
* Compressed messages should be able to sent and consumed correctly
*/
@ParameterizedTest
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}.compression={2}")
@CsvSource(value = Array(
"kraft,none",
"kraft,gzip",
"kraft,snappy",
"kraft,lz4",
"kraft,zstd",
"zk,gzip",
"zk,snappy"
"kraft,classic,none",
"kraft,consumer,none",
"kraft,classic,gzip",
"kraft,consumer,gzip",
"kraft,classic,snappy",
"kraft,consumer,snappy",
"kraft,classic,lz4",
"kraft,consumer,lz4",
"kraft,classic,zstd",
"kraft,consumer,zstd"
))
def testCompression(quorum: String, compression: String): Unit = {
def testCompression(quorum: String, groupProtocol: String, compression: String): Unit = {
val producerProps = new Properties()
val bootstrapServers = TestUtils.plaintextBootstrapServers(Seq(broker))
producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers)
@ -80,7 +84,7 @@ class ProducerCompressionTest extends QuorumTestHarness {
producerProps.put(ProducerConfig.BATCH_SIZE_CONFIG, "66000")
producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "200")
val producer = new KafkaProducer(producerProps, new ByteArraySerializer, new ByteArraySerializer)
val consumer = TestUtils.createConsumer(bootstrapServers)
val consumer = TestUtils.createConsumer(bootstrapServers, GroupProtocol.of(groupProtocol))
try {
// create topic

View File

@ -21,7 +21,7 @@ import java.util.concurrent.ExecutionException
import java.util.Properties
import kafka.integration.KafkaServerTestHarness
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.producer._
import org.apache.kafka.common.config.TopicConfig
import org.apache.kafka.common.errors._
@ -32,7 +32,7 @@ import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, Server
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
class ProducerFailureHandlingTest extends KafkaServerTestHarness {
private val producerBufferSize = 30000
@ -87,8 +87,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
/**
* With ack == 0 the future metadata will have no exceptions with offset -1
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testTooLargeRecordWithAckZero(quorum: String): Unit = {
// create topic
createTopic(topic1, replicationFactor = numServers)
@ -105,8 +105,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
/**
* With ack == 1 the future metadata will throw ExecutionException caused by RecordTooLargeException
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testTooLargeRecordWithAckOne(quorum: String): Unit = {
// create topic
createTopic(topic1, replicationFactor = numServers)
@ -136,14 +136,14 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
/** This should succeed as the replica fetcher thread can handle oversized messages since KIP-74 */
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testPartitionTooLargeForReplicationWithAckAll(quorum: String): Unit = {
checkTooLargeRecordForReplicationWithAckAll(replicaFetchMaxPartitionBytes)
}
/** This should succeed as the replica fetcher thread can handle oversized messages since KIP-74 */
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testResponseTooLargeForReplicationWithAckAll(quorum: String): Unit = {
checkTooLargeRecordForReplicationWithAckAll(replicaFetchMaxResponseBytes)
}
@ -151,8 +151,8 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
/**
* With non-exist-topic the future metadata should return ExecutionException caused by TimeoutException
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testNonExistentTopic(quorum: String): Unit = {
// send a record with non-exist topic
val record = new ProducerRecord(topic2, null, "key".getBytes, "value".getBytes)
@ -170,7 +170,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
* TimeoutException
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testWrongBrokerList(quorum: String): Unit = {
// create topic
createTopic(topic1, replicationFactor = numServers)
@ -188,7 +188,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
* when partition is higher than the upper bound of partitions.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testInvalidPartition(quorum: String): Unit = {
// create topic with a single partition
createTopic(topic1, replicationFactor = numServers)
@ -203,7 +203,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
* The send call after producer closed should throw IllegalStateException
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testSendAfterClosed(quorum: String): Unit = {
// create topic
createTopic(topic1, replicationFactor = numServers)
@ -224,7 +224,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCannotSendToInternalTopic(quorum: String): Unit = {
createOffsetsTopic()
@ -234,7 +234,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testNotEnoughReplicasAfterBrokerShutdown(quorum: String): Unit = {
val topicName = "minisrtest2"
val topicProps = new Properties()

View File

@ -21,7 +21,7 @@ import java.util
import java.util.{Collections, Properties}
import kafka.integration.KafkaServerTestHarness
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.{consumeRecords, createAdminClient}
import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, ProducerState}
import org.apache.kafka.clients.consumer.Consumer
@ -36,7 +36,7 @@ import org.apache.kafka.test.{TestUtils => JTestUtils}
import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import org.opentest4j.AssertionFailedError
import scala.collection.Seq
@ -60,6 +60,7 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness {
override def setUp(testInfo: TestInfo): Unit = {
super.setUp(testInfo)
consumer = TestUtils.createConsumer(bootstrapServers(),
groupProtocolFromTestParameters(),
enableAutoCommit = false,
readCommitted = true)
admin = createAdminClient(brokers, listenerName)
@ -79,9 +80,9 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness {
super.tearDown()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testProducerIdExpirationWithNoTransactions(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testProducerIdExpirationWithNoTransactions(quorum: String, groupProtocol: String): Unit = {
producer = TestUtils.createProducer(bootstrapServers(), enableIdempotence = true)
// Send records to populate producer state cache.
@ -103,9 +104,9 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness {
assertEquals(1, producerState.size)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testTransactionAfterTransactionIdExpiresButProducerIdRemains(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testTransactionAfterTransactionIdExpiresButProducerIdRemains(quorum: String, groupProtocol: String): Unit = {
producer = TestUtils.createTransactionalProducer("transactionalProducer", brokers)
producer.initTransactions()
@ -151,9 +152,9 @@ class ProducerIdExpirationTest extends KafkaServerTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testDynamicProducerIdExpirationMs(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testDynamicProducerIdExpirationMs(quorum: String, groupProtocol: String): Unit = {
producer = TestUtils.createProducer(bootstrapServers(), enableIdempotence = true)
// Send records to populate producer state cache.

View File

@ -50,7 +50,7 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
* succeed as long as the partition is included in the metadata.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testSendWithTopicDeletionMidWay(quorum: String): Unit = {
val numRecords = 10
val topic = "topic"

View File

@ -25,13 +25,13 @@ import org.apache.kafka.common.{KafkaException, TopicPartition}
import org.apache.kafka.common.errors.SaslAuthenticationException
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.api.Assertions._
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.zk.ConfigEntityChangeNotificationZNode
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
import scala.jdk.javaapi.OptionConverters
@ -110,22 +110,25 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
assertThrows(classOf[KafkaException], () => txProducer.initTransactions())
}
@Test
def testConsumerWithAuthenticationFailure(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testConsumerWithAuthenticationFailure(quorum: String, groupProtocol: String): Unit = {
val consumer = createConsumer()
consumer.subscribe(List(topic).asJava)
verifyConsumerWithAuthenticationFailure(consumer)
}
@Test
def testManualAssignmentConsumerWithAuthenticationFailure(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testManualAssignmentConsumerWithAuthenticationFailure(quorum: String, groupProtocol: String): Unit = {
val consumer = createConsumer()
consumer.assign(List(tp).asJava)
verifyConsumerWithAuthenticationFailure(consumer)
}
@Test
def testManualAssignmentConsumerWithAutoCommitDisabledWithAuthenticationFailure(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testManualAssignmentConsumerWithAutoCommitDisabledWithAuthenticationFailure(quorum: String, groupProtocol: String): Unit = {
this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false.toString)
val consumer = createConsumer()
consumer.assign(List(tp).asJava)

View File

@ -16,13 +16,15 @@
*/
package kafka.api
import kafka.utils.TestInfoUtils
import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.common.config.SaslConfigs
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.common.errors.{GroupAuthorizationException, TopicAuthorizationException}
import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout}
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue, fail}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import scala.jdk.CollectionConverters._
@ -56,10 +58,11 @@ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest {
* the second one connects ok, but fails to consume messages due to the ACL.
*/
@Timeout(15)
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testTwoConsumersWithDifferentSaslCredentials(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696"))
def testTwoConsumersWithDifferentSaslCredentials(quorum: String, groupProtocol: String): Unit = {
setAclsAndProduce(tp)
consumerConfig.putIfAbsent(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
val consumer1 = createConsumer()
// consumer2 retrieves its credentials from the static JAAS configuration, so we test also this path

View File

@ -13,12 +13,12 @@
package kafka.api
import kafka.security.JaasTestUtils
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.common.security.auth.SecurityProtocol
import org.apache.kafka.server.config.ZkConfigs
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import scala.jdk.CollectionConverters._
@ -45,9 +45,9 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslSetup {
closeSasl()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testMultipleBrokerMechanisms(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testMultipleBrokerMechanisms(quorum: String, groupProtocol: String): Unit = {
val plainSaslProducer = createProducer()
val plainSaslConsumer = createConsumer()

View File

@ -19,7 +19,7 @@ package kafka.api
import java.util.Properties
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig}
import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback
@ -27,10 +27,10 @@ import org.apache.kafka.common.TopicPartition
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
import org.apache.kafka.server.util.ShutdownableThread
import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import scala.annotation.nowarn
import scala.jdk.CollectionConverters._
@ -77,16 +77,16 @@ class TransactionsBounceTest extends IntegrationTestHarness {
override protected def brokerCount: Int = 4
@nowarn("cat=deprecation")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testWithGroupId(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17961"))
def testWithGroupId(quorum: String, groupProtocol: String): Unit = {
testBrokerFailure((producer, groupId, consumer) =>
producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer).asJava, groupId))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testWithGroupMetadata(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testWithGroupMetadata(quorum: String, groupProtocol: String): Unit = {
testBrokerFailure((producer, _, consumer) =>
producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer).asJava, consumer.groupMetadata()))
}

View File

@ -20,7 +20,7 @@ package kafka.api
import java.util.{Collections, Properties}
import kafka.integration.KafkaServerTestHarness
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.{consumeRecords, createAdminClient}
import org.apache.kafka.clients.admin.{Admin, ProducerState}
import org.apache.kafka.clients.consumer.Consumer
@ -33,7 +33,7 @@ import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, Server
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
import org.junit.jupiter.params.provider.{CsvSource, MethodSource}
import scala.jdk.CollectionConverters._
import scala.collection.Seq
@ -60,6 +60,7 @@ class TransactionsExpirationTest extends KafkaServerTestHarness {
producer = TestUtils.createTransactionalProducer("transactionalProducer", brokers)
consumer = TestUtils.createConsumer(bootstrapServers(),
groupProtocolFromTestParameters(),
enableAutoCommit = false,
readCommitted = true)
admin = createAdminClient(brokers, listenerName)
@ -80,9 +81,9 @@ class TransactionsExpirationTest extends KafkaServerTestHarness {
super.tearDown()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testBumpTransactionalEpochAfterInvalidProducerIdMapping(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testBumpTransactionalEpochAfterInvalidProducerIdMapping(quorum: String, groupProtocol: String): Unit = {
producer.initTransactions()
// Start and then abort a transaction to allow the transactional ID to expire.
@ -120,13 +121,14 @@ class TransactionsExpirationTest extends KafkaServerTestHarness {
}
}
@ParameterizedTest
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}.isTV2Enabled={2}")
@CsvSource(Array(
"zk, false",
"kraft, false",
"kraft, true"
"kraft,classic,false",
"kraft,consumer,false",
"kraft,classic,true",
"kraft,consumer,true",
))
def testTransactionAfterProducerIdExpires(quorum: String, isTV2Enabled: Boolean): Unit = {
def testTransactionAfterProducerIdExpires(quorum: String, groupProtocol: String, isTV2Enabled: Boolean): Unit = {
producer.initTransactions()
// Start and then abort a transaction to allow the producer ID to expire.

View File

@ -17,7 +17,7 @@
package kafka.api
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.{consumeRecords, waitUntilTrue}
import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig, ConsumerGroupMetadata, ConsumerRecord, OffsetAndMetadata}
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
@ -29,7 +29,8 @@ import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, Server
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
import org.junit.jupiter.params.provider.MethodSource
import org.junit.jupiter.params.provider.CsvSource
import java.lang.{Long => JLong}
import java.nio.charset.StandardCharsets
@ -111,9 +112,9 @@ class TransactionsTest extends IntegrationTestHarness {
super.tearDown()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testBasicTransactions(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testBasicTransactions(quorum: String, groupProtocol: String): Unit = {
val producer = transactionalProducers.head
val consumer = transactionalConsumers.head
val unCommittedConsumer = nonTransactionalConsumers.head
@ -172,9 +173,9 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testReadCommittedConsumerShouldNotSeeUndecidedData(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testReadCommittedConsumerShouldNotSeeUndecidedData(quorum: String, groupProtocol: String): Unit = {
val producer1 = transactionalProducers.head
val producer2 = createTransactionalProducer("other")
val readCommittedConsumer = transactionalConsumers.head
@ -240,9 +241,9 @@ class TransactionsTest extends IntegrationTestHarness {
assertNull(readCommittedOffsetsForTimes.get(tp2))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testDelayedFetchIncludesAbortedTransaction(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testDelayedFetchIncludesAbortedTransaction(quorum: String, groupProtocol: String): Unit = {
val producer1 = transactionalProducers.head
val producer2 = createTransactionalProducer("other")
val tp10 = new TopicPartition(topic1, 0)
@ -299,16 +300,16 @@ class TransactionsTest extends IntegrationTestHarness {
}
@nowarn("cat=deprecation")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendOffsetsWithGroupId(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17961"))
def testSendOffsetsWithGroupId(quorum: String, groupProtocol: String): Unit = {
sendOffset((producer, groupId, consumer) =>
producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer).asJava, groupId))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendOffsetsWithGroupMetadata(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendOffsetsWithGroupMetadata(quorum: String, groupProtocol: String): Unit = {
sendOffset((producer, _, consumer) =>
producer.sendOffsetsToTransaction(TestUtils.consumerPositions(consumer).asJava, consumer.groupMetadata()))
}
@ -386,9 +387,9 @@ class TransactionsTest extends IntegrationTestHarness {
assertEquals(valueSeq.size, valueSet.size, s"Expected ${valueSeq.size} unique messages in $topic2.")
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFencingOnCommit(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFencingOnCommit(quorum: String, groupProtocol: String): Unit = {
val producer1 = transactionalProducers(0)
val producer2 = transactionalProducers(1)
val consumer = transactionalConsumers(0)
@ -416,9 +417,9 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFencingOnSendOffsets(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFencingOnSendOffsets(quorum: String, groupProtocol: String): Unit = {
val producer1 = transactionalProducers(0)
val producer2 = transactionalProducers(1)
val consumer = transactionalConsumers(0)
@ -448,9 +449,9 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testOffsetMetadataInSendOffsetsToTransaction(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testOffsetMetadataInSendOffsetsToTransaction(quorum: String, groupProtocol: String): Unit = {
val tp = new TopicPartition(topic1, 0)
val groupId = "group"
@ -474,28 +475,28 @@ class TransactionsTest extends IntegrationTestHarness {
TestUtils.waitUntilTrue(() => offsetAndMetadata.equals(consumer.committed(Set(tp).asJava).get(tp)), "cannot read committed offset")
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testInitTransactionsTimeout(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testInitTransactionsTimeout(quorum: String, groupProtocol: String): Unit = {
testTimeout(needInitAndSendMsg = false, producer => producer.initTransactions())
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testSendOffsetsToTransactionTimeout(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testSendOffsetsToTransactionTimeout(quorum: String, groupProtocol: String): Unit = {
testTimeout(needInitAndSendMsg = true, producer => producer.sendOffsetsToTransaction(
Map(new TopicPartition(topic1, 0) -> new OffsetAndMetadata(0)).asJava, new ConsumerGroupMetadata("test-group")))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testCommitTransactionTimeout(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testCommitTransactionTimeout(quorum: String, groupProtocol: String): Unit = {
testTimeout(needInitAndSendMsg = true, producer => producer.commitTransaction())
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testAbortTransactionTimeout(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testAbortTransactionTimeout(quorum: String, groupProtocol: String): Unit = {
testTimeout(needInitAndSendMsg = true, producer => producer.abortTransaction())
}
@ -514,9 +515,9 @@ class TransactionsTest extends IntegrationTestHarness {
producer.close(Duration.ZERO)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFencingOnSend(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFencingOnSend(quorum: String, groupProtocol: String): Unit = {
val producer1 = transactionalProducers(0)
val producer2 = transactionalProducers(1)
val consumer = transactionalConsumers(0)
@ -559,9 +560,9 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFencingOnAddPartitions(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFencingOnAddPartitions(quorum: String, groupProtocol: String): Unit = {
val producer1 = transactionalProducers(0)
val producer2 = transactionalProducers(1)
val consumer = transactionalConsumers(0)
@ -606,9 +607,9 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFencingOnTransactionExpiration(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFencingOnTransactionExpiration(quorum: String, groupProtocol: String): Unit = {
val producer = createTransactionalProducer("expiringProducer", transactionTimeoutMs = 100)
producer.initTransactions()
@ -649,9 +650,9 @@ class TransactionsTest extends IntegrationTestHarness {
assertTrue(transactionalRecords.isEmpty)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testMultipleMarkersOneLeader(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testMultipleMarkersOneLeader(quorum: String, groupProtocol: String): Unit = {
val firstProducer = transactionalProducers.head
val consumer = transactionalConsumers.head
val unCommittedConsumer = nonTransactionalConsumers.head
@ -687,21 +688,21 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testConsecutivelyRunInitTransactions(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testConsecutivelyRunInitTransactions(quorum: String, groupProtocol: String): Unit = {
val producer = createTransactionalProducer(transactionalId = "normalProducer")
producer.initTransactions()
assertThrows(classOf[IllegalStateException], () => producer.initTransactions())
}
@ParameterizedTest
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@CsvSource(Array(
"zk, false",
"kraft, false"
"kraft,classic,false",
"kraft,consumer,false",
))
def testBumpTransactionalEpochWithTV2Disabled(quorum: String, isTV2Enabled: Boolean): Unit = {
def testBumpTransactionalEpochWithTV2Disabled(quorum: String, groupProtocol: String, isTV2Enabled: Boolean): Unit = {
val producer = createTransactionalProducer("transactionalProducer",
deliveryTimeoutMs = 5000, requestTimeoutMs = 5000)
val consumer = transactionalConsumers.head
@ -761,9 +762,9 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@CsvSource(Array("kraft, true"))
def testBumpTransactionalEpochWithTV2Enabled(quorum: String, isTV2Enabled: Boolean): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@CsvSource(Array("kraft, classic, true", "kraft, consumer, true"))
def testBumpTransactionalEpochWithTV2Enabled(quorum: String, groupProtocol: String, isTV2Enabled: Boolean): Unit = {
val producer = createTransactionalProducer("transactionalProducer",
deliveryTimeoutMs = 5000, requestTimeoutMs = 5000)
val consumer = transactionalConsumers.head
@ -848,13 +849,14 @@ class TransactionsTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}.isTV2Enabled={2}")
@CsvSource(Array(
"zk, false",
"kraft, false",
"kraft, true"
"kraft, classic, false",
"kraft, consumer, false",
"kraft, classic, true",
"kraft, consumer, true",
))
def testFailureToFenceEpoch(quorum: String, isTV2Enabled: Boolean): Unit = {
def testFailureToFenceEpoch(quorum: String, groupProtocol: String, isTV2Enabled: Boolean): Unit = {
val producer1 = transactionalProducers.head
val producer2 = createTransactionalProducer("transactional-producer", maxBlockMs = 1000)
val initialProducerEpoch = 0
@ -935,6 +937,7 @@ class TransactionsTest extends IntegrationTestHarness {
maxPollRecords: Int = 500,
props: Properties = new Properties) = {
val consumer = TestUtils.createConsumer(bootstrapServers(),
groupProtocolFromTestParameters(),
groupId = group,
enableAutoCommit = false,
readCommitted = true,
@ -945,6 +948,7 @@ class TransactionsTest extends IntegrationTestHarness {
private def createReadUncommittedConsumer(group: String) = {
val consumer = TestUtils.createConsumer(bootstrapServers(),
groupProtocolFromTestParameters(),
groupId = group,
enableAutoCommit = false)
nonTransactionalConsumers += consumer

View File

@ -20,17 +20,17 @@ package kafka.api
import java.util.Properties
import kafka.integration.KafkaServerTestHarness
import kafka.server.KafkaConfig
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.TestUtils.consumeRecords
import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.clients.producer.KafkaProducer
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.coordinator.transaction.{TransactionLogConfig, TransactionStateManagerConfig}
import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs, ServerLogConfigs}
import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import scala.collection.{Seq, mutable}
import scala.jdk.CollectionConverters._
@ -72,9 +72,9 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness {
super.tearDown()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testTransactionalProducerSingleBrokerMaxInFlightOne(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testTransactionalProducerSingleBrokerMaxInFlightOne(quorum: String, groupProtocol: String): Unit = {
// We want to test with one broker to verify multiple requests queued on a connection
assertEquals(1, brokers.size)
@ -120,6 +120,7 @@ class TransactionsWithMaxInFlightOneTest extends KafkaServerTestHarness {
private def createReadCommittedConsumer(group: String) = {
val consumer = TestUtils.createConsumer(bootstrapServers(),
groupProtocolFromTestParameters(),
groupId = group,
enableAutoCommit = false,
readCommitted = true)

View File

@ -82,7 +82,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDynamicConnectionQuota(quorum: String): Unit = {
val maxConnectionsPerIP = 5
@ -110,7 +110,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDynamicListenerConnectionQuota(quorum: String): Unit = {
val initialConnectionCount = connectionCount
@ -183,7 +183,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDynamicListenerConnectionCreationRateQuota(quorum: String): Unit = {
// Create another listener. PLAINTEXT is an inter-broker listener
// keep default limits
@ -245,7 +245,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDynamicIpConnectionRateQuota(quorum: String): Unit = {
val connRateLimit = 10
val initialConnectionCount = connectionCount

View File

@ -67,7 +67,7 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDynamicNumNetworkThreads(quorum: String): Unit = {
// Increase the base network thread count
val newBaseNetworkThreadsCount = SocketServerConfigs.NUM_NETWORK_THREADS_DEFAULT + 1

View File

@ -71,9 +71,9 @@ import org.apache.kafka.server.util.ShutdownableThread
import org.apache.kafka.storage.internals.log.{CleanerConfig, LogConfig}
import org.apache.kafka.test.{TestSslUtils, TestUtils => JTestUtils}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, Test, TestInfo}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.{CsvSource, MethodSource}
import java.util.concurrent.atomic.AtomicInteger
import scala.annotation.nowarn
@ -185,9 +185,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
closeSasl()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testConfigDescribeUsingAdminClient(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testConfigDescribeUsingAdminClient(quorum: String, groupProtocol: String): Unit = {
def verifyConfig(configName: String, configEntry: ConfigEntry, isSensitive: Boolean, isReadOnly: Boolean,
expectedProps: Properties): Unit = {
@ -285,9 +285,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
assertEquals(List((CleanerConfig.LOG_CLEANER_THREADS_PROP, ConfigSource.DEFAULT_CONFIG)), synonymsList(logCleanerThreads))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testUpdatesUsingConfigProvider(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testUpdatesUsingConfigProvider(quorum: String, groupProtocol: String): Unit = {
val PollingIntervalVal = f"$${file:polling.interval:interval}"
val PollingIntervalUpdateVal = f"$${file:polling.interval:updinterval}"
val SslTruststoreTypeVal = f"$${file:ssl.truststore.type:storetype}"
@ -358,15 +358,15 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testKeyStoreAlter(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testKeyStoreAlter(quorum: String, groupProtocol: String): Unit = {
val topic2 = "testtopic2"
TestUtils.createTopicWithAdmin(adminClients.head, topic2, servers, controllerServers, numPartitions, replicationFactor = numServers)
// Start a producer and consumer that work with the current broker keystore.
// This should continue working while changes are made
val (producerThread, consumerThread) = startProduceConsume(retries = 0)
val (producerThread, consumerThread) = startProduceConsume(retries = 0, groupProtocol)
TestUtils.waitUntilTrue(() => consumerThread.received >= 10, "Messages not received")
// Producer with new truststore should fail to connect before keystore update
@ -384,7 +384,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val producer = ProducerBuilder().trustStoreProps(sslProperties2).maxRetries(0).build()
// Start the new consumer in a separate group than the continuous consumer started at the beginning of the test so
// that it is not disrupted by rebalance.
val consumer = ConsumerBuilder("group2").trustStoreProps(sslProperties2).topic(topic2).build()
val consumer = ConsumerBuilder("group2", groupProtocol).trustStoreProps(sslProperties2).topic(topic2).build()
verifyProduceConsume(producer, consumer, 10, topic2)
// Broker keystore update for internal listener with incompatible keystore should fail without update
@ -427,9 +427,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
stopAndVerifyProduceConsume(producerThread, consumerThread)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testTrustStoreAlter(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testTrustStoreAlter(quorum: String, groupProtocol: String): Unit = {
val producerBuilder = ProducerBuilder().listenerName(SecureInternal).securityProtocol(SecurityProtocol.SSL)
// Producer with new keystore should fail to connect before truststore update
@ -450,7 +450,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
def verifySslProduceConsume(keyStoreProps: Properties, group: String): Unit = {
val producer = producerBuilder.keyStoreProps(keyStoreProps).build()
val consumer = ConsumerBuilder(group)
val consumer = ConsumerBuilder(group, groupProtocol)
.listenerName(SecureInternal)
.securityProtocol(SecurityProtocol.SSL)
.keyStoreProps(keyStoreProps)
@ -532,10 +532,10 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testLogCleanerConfig(quorum: String): Unit = {
val (producerThread, consumerThread) = startProduceConsume(retries = 0)
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testLogCleanerConfig(quorum: String, groupProtocol: String): Unit = {
val (producerThread, consumerThread) = startProduceConsume(retries = 0, groupProtocol)
verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 1)
@ -585,9 +585,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
verifyThreads("kafka-log-cleaner-thread-", countPerBroker = 2)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testConsecutiveConfigChange(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testConsecutiveConfigChange(quorum: String, groupProtocol: String): Unit = {
val topic2 = "testtopic2"
val topicProps = new Properties
topicProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "2")
@ -629,10 +629,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
assertEquals("2", log.config.originals().get(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG).toString) // Verify topic-level config still survives
}
@Test
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
@nowarn("cat=deprecation") // See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for deprecation details
def testDefaultTopicConfig(): Unit = {
val (producerThread, consumerThread) = startProduceConsume(retries = 0)
def testDefaultTopicConfig(quorum: String, groupProtocol: String): Unit = {
val (producerThread, consumerThread) = startProduceConsume(retries = 0, groupProtocol)
val props = new Properties
props.put(ServerLogConfigs.LOG_SEGMENT_BYTES_CONFIG, "4000")
@ -751,8 +752,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
@Test
def testUncleanLeaderElectionEnable(): Unit = {
@ParameterizedTest(name = "{displayName}.groupProtocol={0}")
@CsvSource(Array("classic, consumer"))
def testUncleanLeaderElectionEnable(groupProtocol: String): Unit = {
val controller = servers.find(_.config.brokerId == TestUtils.waitUntilControllerElected(zkClient)).get
val controllerId = controller.config.brokerId
@ -762,7 +764,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
TestUtils.createTopic(zkClient, topic, assignment, servers)
val producer = ProducerBuilder().acks(1).build()
val consumer = ConsumerBuilder("unclean-leader-test").enableAutoCommit(false).topic(topic).build()
val consumer = ConsumerBuilder("unclean-leader-test", groupProtocol).enableAutoCommit(false).topic(topic).build()
verifyProduceConsume(producer, consumer, numRecords = 10, topic)
consumer.commitSync()
@ -812,8 +814,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
consumer.commitSync()
}
@Test
def testThreadPoolResize(): Unit = {
@ParameterizedTest(name = "{displayName}.groupProtocol={0}")
@CsvSource(Array("classic, consumer"))
def testThreadPoolResize(groupProtocol: String): Unit = {
val requestHandlerPrefix = "data-plane-kafka-request-handler-"
val networkThreadPrefix = "data-plane-kafka-network-thread-"
val fetcherThreadPrefix = "ReplicaFetcherThread-"
@ -866,7 +869,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
def verifyThreadPoolResize(propName: String, currentSize: => Int, threadPrefix: String, mayReceiveDuplicates: Boolean): Unit = {
maybeVerifyThreadPoolSize(currentSize, threadPrefix)
val numRetries = if (mayReceiveDuplicates) 100 else 0
val (producerThread, consumerThread) = startProduceConsume(retries = numRetries)
val (producerThread, consumerThread) = startProduceConsume(retries = numRetries, groupProtocol)
var threadPoolSize = currentSize
(1 to 2).foreach { _ =>
threadPoolSize = reducePoolSize(propName, threadPoolSize, threadPrefix)
@ -950,8 +953,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
@Test
def testMetricsReporterUpdate(): Unit = {
@ParameterizedTest(name = "{displayName}.groupProtocol={0}")
@CsvSource(Array("classic, consumer"))
def testMetricsReporterUpdate(groupProtocol: String): Unit = {
// Add a new metrics reporter
val newProps = new Properties
newProps.put(TestMetricsReporter.PollingIntervalProp, "100")
@ -971,7 +975,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
server.quotaManagers.produce.updateQuota(None, Some(clientId), Some(clientId),
Some(Quota.upperBound(10000000)))
}
val (producerThread, consumerThread) = startProduceConsume(retries = 0, clientId)
val (producerThread, consumerThread) = startProduceConsume(retries = 0, groupProtocol, clientId)
TestUtils.waitUntilTrue(() => consumerThread.received >= 5, "Messages not sent")
// Verify that JMX reporter is still active (test a metric registered after the dynamic reporter update)
@ -1037,9 +1041,10 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
stopAndVerifyProduceConsume(producerThread, consumerThread)
}
@Test
@ParameterizedTest(name = "{displayName}.groupProtocol={0}")
@CsvSource(Array("classic"))
// Modifying advertised listeners is not supported in KRaft
def testAdvertisedListenerUpdate(): Unit = {
def testAdvertisedListenerUpdate(groupProtocol: String): Unit = {
val adminClient = adminClients.head
val externalAdminClient = createAdminClient(SecurityProtocol.SASL_SSL, SecureExternal)
@ -1097,7 +1102,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val topic2 = "testtopic2"
TestUtils.createTopic(zkClient, topic2, numPartitions, replicationFactor = numServers, servers)
val producer = ProducerBuilder().trustStoreProps(sslProperties1).maxRetries(0).build()
val consumer = ConsumerBuilder("group2").trustStoreProps(sslProperties1).topic(topic2).build()
val consumer = ConsumerBuilder("group2", groupProtocol).trustStoreProps(sslProperties1).topic(topic2).build()
verifyProduceConsume(producer, consumer, 10, topic2)
// Verify updating inter-broker listener
@ -1108,10 +1113,11 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
servers.foreach(server => assertEquals(SecureInternal, server.config.interBrokerListenerName.value))
}
@Test
@ParameterizedTest(name = "{displayName}.groupProtocol={0}")
@CsvSource(Array("classic, consumer"))
@Disabled // Re-enable once we make it less flaky (KAFKA-6824)
def testAddRemoveSslListener(): Unit = {
verifyAddListener("SSL", SecurityProtocol.SSL, Seq.empty)
def testAddRemoveSslListener(groupProtocol: String): Unit = {
verifyAddListener("SSL", SecurityProtocol.SSL, Seq.empty, groupProtocol)
// Restart servers and check secret rotation
servers.foreach(_.shutdown())
@ -1149,13 +1155,14 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
verifyListener(SecurityProtocol.SSL, None, "add-ssl-listener-group2")
verifyListener(SecurityProtocol.SSL, None, "add-ssl-listener-group2", groupProtocol)
createAdminClient(SecurityProtocol.SSL, SecureInternal)
verifyRemoveListener("SSL", SecurityProtocol.SSL, Seq.empty)
verifyRemoveListener("SSL", SecurityProtocol.SSL, Seq.empty, groupProtocol)
}
@Test
def testAddRemoveSaslListeners(): Unit = {
@ParameterizedTest(name = "{displayName}.groupProtocol={0}")
@CsvSource(Array("classic, consumer"))
def testAddRemoveSaslListeners(groupProtocol: String): Unit = {
createScramCredentials(adminClients.head, JaasTestUtils.KAFKA_SCRAM_USER, JaasTestUtils.KAFKA_SCRAM_PASSWORD)
createScramCredentials(adminClients.head, JaasTestUtils.KAFKA_SCRAM_ADMIN, JaasTestUtils.KAFKA_SCRAM_ADMIN_PASSWORD)
initializeKerberos()
@ -1170,9 +1177,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}}
//verifyAddListener("SASL_SSL", SecurityProtocol.SASL_SSL, Seq("SCRAM-SHA-512", "SCRAM-SHA-256", "PLAIN"))
verifyAddListener("SASL_PLAINTEXT", SecurityProtocol.SASL_PLAINTEXT, Seq("GSSAPI"))
verifyAddListener("SASL_PLAINTEXT", SecurityProtocol.SASL_PLAINTEXT, Seq("GSSAPI"), groupProtocol)
//verifyRemoveListener("SASL_SSL", SecurityProtocol.SASL_SSL, Seq("SCRAM-SHA-512", "SCRAM-SHA-256", "PLAIN"))
verifyRemoveListener("SASL_PLAINTEXT", SecurityProtocol.SASL_PLAINTEXT, Seq("GSSAPI"))
verifyRemoveListener("SASL_PLAINTEXT", SecurityProtocol.SASL_PLAINTEXT, Seq("GSSAPI"), groupProtocol)
// Verify that a listener added to a subset of servers doesn't cause any issues
// when metadata is processed by the client.
@ -1188,9 +1195,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
assertTrue(partitions.exists(_.leader == null), "Did not find partitions with no leader")
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testReconfigureRemovedListener(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testReconfigureRemovedListener(quorum: String, groupProtocol: String): Unit = {
val client = adminClients.head
val broker = servers.head
assertEquals(2, broker.config.dynamicConfig.reconfigurables.asScala.count(r => r.isInstanceOf[DataPlaneAcceptor]))
@ -1268,9 +1275,9 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testTransactionVerificationEnable(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testTransactionVerificationEnable(quorum: String, groupProtocol: String): Unit = {
def verifyConfiguration(enabled: Boolean): Unit = {
servers.foreach { server =>
TestUtils.waitUntilTrue(() => server.logManager.producerStateManagerConfig.transactionVerificationEnabled == enabled, "Configuration was not updated.")
@ -1301,27 +1308,29 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
private def verifyAddListener(listenerName: String, securityProtocol: SecurityProtocol,
saslMechanisms: Seq[String]): Unit = {
saslMechanisms: Seq[String],
groupProtocol: String): Unit = {
addListener(servers, listenerName, securityProtocol, saslMechanisms)
TestUtils.waitUntilTrue(() => servers.forall(hasListenerMetric(_, listenerName)),
"Processors not started for new listener")
if (saslMechanisms.nonEmpty)
saslMechanisms.foreach { mechanism =>
verifyListener(securityProtocol, Some(mechanism), s"add-listener-group-$securityProtocol-$mechanism")
verifyListener(securityProtocol, Some(mechanism), s"add-listener-group-$securityProtocol-$mechanism", groupProtocol)
}
else
verifyListener(securityProtocol, None, s"add-listener-group-$securityProtocol")
verifyListener(securityProtocol, None, s"add-listener-group-$securityProtocol", groupProtocol)
}
private def verifyRemoveListener(listenerName: String, securityProtocol: SecurityProtocol,
saslMechanisms: Seq[String]): Unit = {
saslMechanisms: Seq[String],
groupProtocol: String): Unit = {
val saslMechanism = if (saslMechanisms.isEmpty) "" else saslMechanisms.head
val producer1 = ProducerBuilder().listenerName(listenerName)
.securityProtocol(securityProtocol)
.saslMechanism(saslMechanism)
.maxRetries(1000)
.build()
val consumer1 = ConsumerBuilder(s"remove-listener-group-$securityProtocol")
val consumer1 = ConsumerBuilder(s"remove-listener-group-$securityProtocol", groupProtocol)
.listenerName(listenerName)
.securityProtocol(securityProtocol)
.saslMechanism(saslMechanism)
@ -1365,7 +1374,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
val topic2 = "testtopic2"
TestUtils.createTopic(zkClient, topic2, numPartitions, replicationFactor = numServers, servers)
val producer2 = ProducerBuilder().trustStoreProps(sslProperties1).maxRetries(0).build()
val consumer2 = ConsumerBuilder(s"remove-listener-group2-$securityProtocol")
val consumer2 = ConsumerBuilder(s"remove-listener-group2-$securityProtocol", groupProtocol)
.trustStoreProps(sslProperties1)
.topic(topic2)
.autoOffsetReset("latest")
@ -1377,7 +1386,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
verifyTimeout(consumerFuture)
}
private def verifyListener(securityProtocol: SecurityProtocol, saslMechanism: Option[String], groupId: String): Unit = {
private def verifyListener(securityProtocol: SecurityProtocol, saslMechanism: Option[String], groupId: String, groupProtocol: String): Unit = {
val mechanism = saslMechanism.getOrElse("")
val retries = 1000 // since it may take time for metadata to be updated on all brokers
val producer = ProducerBuilder().listenerName(securityProtocol.name)
@ -1385,7 +1394,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
.saslMechanism(mechanism)
.maxRetries(retries)
.build()
val consumer = ConsumerBuilder(groupId)
val consumer = ConsumerBuilder(groupId, groupProtocol)
.listenerName(securityProtocol.name)
.securityProtocol(securityProtocol)
.saslMechanism(mechanism)
@ -1686,10 +1695,10 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
assertTrue(resized, s"Invalid threads: expected $expectedCount, got ${threads.size}: $threads")
}
private def startProduceConsume(retries: Int, producerClientId: String = "test-producer"): (ProducerThread, ConsumerThread) = {
private def startProduceConsume(retries: Int, groupProtocol: String, producerClientId: String = "test-producer"): (ProducerThread, ConsumerThread) = {
val producerThread = new ProducerThread(producerClientId, retries)
clientThreads += producerThread
val consumerThread = new ConsumerThread(producerThread)
val consumerThread = new ConsumerThread(producerThread, groupProtocol)
clientThreads += consumerThread
consumerThread.start()
producerThread.start()
@ -1834,7 +1843,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
private case class ConsumerBuilder(group: String) extends ClientBuilder[Consumer[String, String]] {
private case class ConsumerBuilder(group: String, groupProtocol: String) extends ClientBuilder[Consumer[String, String]] {
private var _autoOffsetReset = "earliest"
private var _enableAutoCommit = false
private var _topic = DynamicBrokerReconfigurationTest.this.topic
@ -1849,6 +1858,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, _autoOffsetReset)
consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, group)
consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, _enableAutoCommit.toString)
consumerProps.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
val consumer = new KafkaConsumer[String, String](consumerProps, new StringDeserializer, new StringDeserializer)
consumers += consumer
@ -1882,8 +1892,8 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
}
}
private class ConsumerThread(producerThread: ProducerThread) extends ShutdownableThread("test-consumer", false) {
private val consumer = ConsumerBuilder("group1").enableAutoCommit(true).build()
private class ConsumerThread(producerThread: ProducerThread, groupProtocol: String) extends ShutdownableThread("test-consumer", false) {
private val consumer = ConsumerBuilder("group1", groupProtocol).enableAutoCommit(true).build()
val lastReceived = new ConcurrentHashMap[Int, Int]()
val missingRecords = new ConcurrentLinkedQueue[Int]()
@volatile var outOfOrder = false

View File

@ -16,7 +16,7 @@
*/
package kafka.server
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.admin.NewPartitionReassignment
import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer, RangeAssignor}
import org.apache.kafka.clients.producer.ProducerRecord
@ -29,7 +29,7 @@ import org.apache.kafka.server.config.ServerLogConfigs
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{Disabled, Timeout}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import java.util
import java.util.{Collections, Properties}
@ -56,10 +56,10 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
.map(KafkaConfig.fromProps(_, overridingProps))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
@Timeout(15)
def testFollowerCompleteDelayedFetchesOnReplication(quorum: String): Unit = {
def testFollowerCompleteDelayedFetchesOnReplication(quorum: String, groupProtocol: String): Unit = {
// Create a topic with 2 replicas where broker 0 is the leader and 1 is the follower.
val admin = createAdminClient()
val partitionLeaders = TestUtils.createTopicWithAdmin(
@ -101,9 +101,9 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFetchFromLeaderWhilePreferredReadReplicaIsUnavailable(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFetchFromLeaderWhilePreferredReadReplicaIsUnavailable(quorum: String, groupProtocol: String): Unit = {
// Create a topic with 2 replicas where broker 0 is the leader and 1 is the follower.
val admin = createAdminClient()
TestUtils.createTopicWithAdmin(
@ -129,9 +129,9 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
assertEquals(-1, getPreferredReplica)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testFetchFromFollowerWithRoll(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testFetchFromFollowerWithRoll(quorum: String, groupProtocol: String): Unit = {
// Create a topic with 2 replicas where broker 0 is the leader and 1 is the follower.
val admin = createAdminClient()
TestUtils.createTopicWithAdmin(
@ -148,6 +148,7 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group")
consumerProps.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest")
consumerProps.put(ConsumerConfig.CLIENT_RACK_CONFIG, followerBrokerId.toString)
consumerProps.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol)
val consumer = new KafkaConsumer(consumerProps, new ByteArrayDeserializer, new ByteArrayDeserializer)
try {
consumer.subscribe(List(topic).asJava)
@ -181,9 +182,9 @@ class FetchFromFollowerIntegrationTest extends BaseFetchRequestTest {
}
@Disabled
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testRackAwareRangeAssignor(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testRackAwareRangeAssignor(quorum: String, groupProtocol: String): Unit = {
val partitionList = brokers.indices.toList
val topicWithAllPartitionsOnAllRacks = "topicWithAllPartitionsOnAllRacks"

View File

@ -19,15 +19,15 @@ package kafka.server
import java.time.Duration
import java.util.Arrays.asList
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV0, IBP_2_8_IV1, IBP_3_1_IV0}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import scala.collection.{Map, Seq}
@ -43,15 +43,17 @@ class FetchRequestBetweenDifferentIbpTest extends BaseRequestTest {
)
}
@Test
def testControllerOldIBP(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testControllerOldIBP(quorum: String, groupProtocol: String): Unit = {
// Ensure controller version < IBP_2_8_IV1, and then create a topic where leader of partition 0 is not the controller,
// leader of partition 1 is.
testControllerWithGivenIBP(IBP_2_7_IV0, 0)
}
@Test
def testControllerNewIBP(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testControllerNewIBP(quorum: String, groupProtocol: String): Unit = {
// Ensure controller version = IBP_3_1_IV0, and then create a topic where leader of partition 1 is the old version.
testControllerWithGivenIBP(IBP_3_1_IV0, 2)
}
@ -79,13 +81,15 @@ class FetchRequestBetweenDifferentIbpTest extends BaseRequestTest {
assertEquals(2, count)
}
@Test
def testControllerNewToOldIBP(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testControllerNewToOldIBP(quorum: String, groupProtocol: String): Unit = {
testControllerSwitchingIBP(IBP_3_1_IV0, 2, IBP_2_7_IV0, 0)
}
@Test
def testControllerOldToNewIBP(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testControllerOldToNewIBP(quorum: String, groupProtocol: String): Unit = {
testControllerSwitchingIBP(IBP_2_7_IV0, 0, IBP_3_1_IV0, 2)
}

View File

@ -19,8 +19,7 @@ package kafka.server
import java.time.Duration
import java.util.Arrays.asList
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.zk.ZkVersion
import org.apache.kafka.clients.producer.ProducerRecord
import org.apache.kafka.common.TopicPartition
@ -28,7 +27,8 @@ import org.apache.kafka.server.config.ReplicationConfigs
import org.apache.kafka.server.common.MetadataVersion
import org.apache.kafka.server.common.MetadataVersion.{IBP_2_7_IV0, IBP_3_1_IV0}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.Test
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import scala.collection.{Map, Seq}
@ -43,8 +43,9 @@ class FetchRequestTestDowngrade extends BaseRequestTest {
)
}
@Test
def testTopicIdIsRemovedFromFetcherWhenControllerDowngrades(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testTopicIdIsRemovedFromFetcherWhenControllerDowngrades(quorum: String, groupProtocol: String): Unit = {
val tp = new TopicPartition("topic", 0)
val producer = createProducer()
val consumer = createConsumer()

View File

@ -24,7 +24,7 @@ import java.util.{Collections, Properties}
import java.util.concurrent.{CountDownLatch, Executors, TimeUnit}
import javax.security.auth.login.LoginContext
import kafka.api.{Both, IntegrationTestHarness, SaslSetup}
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.config.SaslConfigs
@ -39,6 +39,8 @@ import org.apache.kafka.common.utils.{LogContext, MockTime}
import org.apache.kafka.network.SocketServerConfigs
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import scala.jdk.CollectionConverters._
@ -174,8 +176,9 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
* Test that when client fails to verify authenticity of the server, the resulting failed authentication exception
* is thrown immediately, and is not affected by <code>connection.failed.authentication.delay.ms</code>.
*/
@Test
def testServerAuthenticationFailure(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testServerAuthenticationFailure(quorum: String, groupProtocol: String): Unit = {
// Setup client with a non-existent service principal, so that server authentication fails on the client
val clientLoginContext = jaasClientLoginModule(kafkaClientSaslMechanism, Some("another-kafka-service"))
val configOverrides = new Properties()

View File

@ -23,7 +23,7 @@ import java.util.concurrent.TimeUnit
import kafka.api.SaslSetup
import kafka.security.JaasTestUtils
import kafka.security.JaasTestUtils.JaasSection
import kafka.utils.TestUtils
import kafka.utils.{TestInfoUtils, TestUtils}
import kafka.utils.Implicits._
import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
@ -35,7 +35,9 @@ import org.apache.kafka.server.config.{ReplicationConfigs, ZkConfigs}
import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
import org.apache.kafka.network.SocketServerConfigs
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.MethodSource
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
@ -134,7 +136,8 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT
producers(clientMetadata) = TestUtils.createProducer(bootstrapServers, acks = -1,
securityProtocol = endPoint.securityProtocol, trustStoreFile = trustStoreFile, saslProperties = saslProps)
consumers(clientMetadata) = TestUtils.createConsumer(bootstrapServers, groupId = clientMetadata.toString,
consumers(clientMetadata) = TestUtils.createConsumer(bootstrapServers,
groupProtocolFromTestParameters(), groupId = clientMetadata.toString,
securityProtocol = endPoint.securityProtocol, trustStoreFile = trustStoreFile, saslProperties = saslProps)
}
@ -161,8 +164,9 @@ abstract class MultipleListenersWithSameSecurityProtocolBaseTest extends QuorumT
* Tests that we can produce and consume to/from all broker-defined listeners and security protocols. We produce
* with acks=-1 to ensure that replication is also working.
*/
@Test
def testProduceConsume(): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit"))
def testProduceConsume(quorum: String, groupProtocol: String): Unit = {
producers.foreach { case (clientMetadata, producer) =>
val producerRecords = (1 to 10).map(i => new ProducerRecord(clientMetadata.topic, s"key$i".getBytes,
s"value$i".getBytes))

View File

@ -22,7 +22,7 @@ import kafka.controller.ControllerEventManager
import java.io.File
import java.net.InetSocketAddress
import java.util
import java.util.{Collections, Optional, OptionalInt, Properties}
import java.util.{Collections, Locale, Optional, OptionalInt, Properties, stream}
import java.util.concurrent.{CompletableFuture, TimeUnit}
import javax.security.auth.login.Configuration
import kafka.utils.{CoreUtils, Logging, TestInfoUtils, TestUtils}
@ -51,6 +51,7 @@ import org.apache.zookeeper.client.ZKClientConfig
import org.apache.zookeeper.{WatchedEvent, Watcher, ZooKeeper}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{AfterAll, AfterEach, BeforeAll, BeforeEach, Tag, TestInfo}
import org.junit.jupiter.params.provider.Arguments
import java.nio.file.{Files, Paths}
import scala.collection.Seq
@ -204,10 +205,19 @@ abstract class QuorumTestHarness extends Logging {
TestInfoUtils.isShareGroupTest(testInfo)
}
def maybeGroupProtocolSpecified(testInfo: TestInfo): Option[GroupProtocol] = {
def maybeGroupProtocolSpecified(): Option[GroupProtocol] = {
TestInfoUtils.maybeGroupProtocolSpecified(testInfo)
}
def groupProtocolFromTestParameters(): GroupProtocol = {
val gp = maybeGroupProtocolSpecified()
if (gp.isEmpty)
throw new IllegalStateException("Please specify the \"groupProtocol\" parameter when writing the test")
gp.get
}
def checkIsZKTest(): Unit = {
if (isKRaftTest()) {
throw new RuntimeException("This function can't be accessed when running the test " +
@ -531,4 +541,41 @@ object QuorumTestHarness {
s"Found ${unexpected.size} unexpected threads during $context: " +
s"${unexpected.mkString("`", ",", "`")}")
}
// We want to test the following combinations:
// * KRaft and the classic group protocol
// * KRaft and the consumer group protocol
def getTestQuorumAndGroupProtocolParametersAll: java.util.stream.Stream[Arguments] = {
stream.Stream.of(
Arguments.of("kraft", GroupProtocol.CLASSIC.name.toLowerCase(Locale.ROOT)),
Arguments.of("kraft", GroupProtocol.CONSUMER.name.toLowerCase(Locale.ROOT))
)
}
// For tests that only work with the classic group protocol, we want to test the following combinations:
// * KRaft and the classic group protocol
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly: java.util.stream.Stream[Arguments] = {
stream.Stream.of(
Arguments.of("kraft", GroupProtocol.CLASSIC.name.toLowerCase(Locale.ROOT))
)
}
// For tests that only work with the consumer group protocol, we want to test the following combination:
// * KRaft and the consumer group protocol
def getTestQuorumAndGroupProtocolParametersConsumerGroupProtocolOnly: stream.Stream[Arguments] = {
stream.Stream.of(
Arguments.of("kraft", GroupProtocol.CONSUMER.name.toLowerCase(Locale.ROOT))
)
}
// The following is for tests that only work with the classic group protocol because of relying on Zookeeper
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit: java.util.stream.Stream[Arguments] = stream.Stream.of(Arguments.of("zk", GroupProtocol.CLASSIC.name.toLowerCase(Locale.ROOT)))
// The following parameter groups are to *temporarily* avoid bugs with the CONSUMER group protocol Consumer
// implementation that would otherwise cause tests to fail.
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_16176: stream.Stream[Arguments] = getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17696: stream.Stream[Arguments] = getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17960: stream.Stream[Arguments] = getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17961: stream.Stream[Arguments] = getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
def getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_KAFKA_17964: stream.Stream[Arguments] = getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly
}

View File

@ -71,7 +71,7 @@ class AddPartitionsTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testWrongReplicaCount(quorum: String): Unit = {
assertEquals(classOf[InvalidReplicaAssignmentException], assertThrows(classOf[ExecutionException], () => {
admin.createPartitions(Collections.singletonMap(topic1,
@ -84,7 +84,7 @@ class AddPartitionsTest extends BaseRequestTest {
* and consecutive.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testMissingPartitionsInCreateTopics(quorum: String): Unit = {
val topic6Placements = new util.HashMap[Integer, util.List[Integer]]
topic6Placements.put(1, asList(0, 1))
@ -110,7 +110,7 @@ class AddPartitionsTest extends BaseRequestTest {
* enough partitions.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testMissingPartitionsInCreatePartitions(quorum: String): Unit = {
val cause = assertThrows(classOf[ExecutionException], () =>
admin.createPartitions(Collections.singletonMap(topic1,
@ -134,7 +134,7 @@ class AddPartitionsTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testIncrementPartitions(quorum: String): Unit = {
admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(3))).all().get()
@ -163,7 +163,7 @@ class AddPartitionsTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testManualAssignmentOfReplicas(quorum: String): Unit = {
// Add 2 partitions
admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3,

View File

@ -1672,7 +1672,7 @@ class PartitionTest extends AbstractPartitionTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = {
val kraft = quorum == "kraft"

View File

@ -70,7 +70,7 @@ class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with
* checking all metrics we care in a single test is faster though it would be more elegant to have 3 @Test methods
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testMetricsDuringTopicCreateDelete(quorum: String): Unit = {
// For UnderReplicatedPartitions, because of https://issues.apache.org/jira/browse/KAFKA-4605

View File

@ -32,7 +32,7 @@ class MinIsrConfigTest extends KafkaServerTestHarness {
def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnectOrNull).map(KafkaConfig.fromProps(_, overridingProps))
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDefaultKafkaConfig(quorum: String): Unit = {
assert(brokers.head.logManager.initialDefaultConfig.minInSyncReplicas == 5)
}

View File

@ -23,7 +23,7 @@ import scala.util.Random
import scala.jdk.CollectionConverters._
import scala.collection.{Map, Seq}
import kafka.server.{KafkaBroker, KafkaConfig, MetadataCache, QuorumTestHarness}
import kafka.utils.{CoreUtils, TestUtils}
import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils}
import kafka.utils.TestUtils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
@ -38,7 +38,7 @@ import org.apache.log4j.{Level, Logger}
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import com.yammer.metrics.core.Meter
import org.apache.kafka.metadata.LeaderConstants
@ -119,9 +119,9 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
admin = TestUtils.createAdminClient(brokers, ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT), adminConfigs)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testUncleanLeaderElectionEnabled(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testUncleanLeaderElectionEnabled(quorum: String, groupProtocol: String): Unit = {
// enable unclean leader election
configProps1.put("unclean.leader.election.enable", "true")
configProps2.put("unclean.leader.election.enable", "true")
@ -132,9 +132,9 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
verifyUncleanLeaderElectionEnabled()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testUncleanLeaderElectionDisabled(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testUncleanLeaderElectionDisabled(quorum: String, groupProtocol: String): Unit = {
// unclean leader election is disabled by default
startBrokers(Seq(configProps1, configProps2))
@ -144,9 +144,9 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
verifyUncleanLeaderElectionDisabled()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testUncleanLeaderElectionEnabledByTopicOverride(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testUncleanLeaderElectionEnabledByTopicOverride(quorum: String, groupProtocol: String): Unit = {
// disable unclean leader election globally, but enable for our specific test topic
configProps1.put("unclean.leader.election.enable", "false")
configProps2.put("unclean.leader.election.enable", "false")
@ -160,9 +160,9 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
verifyUncleanLeaderElectionEnabled()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testUncleanLeaderElectionDisabledByTopicOverride(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testUncleanLeaderElectionDisabledByTopicOverride(quorum: String, groupProtocol: String): Unit = {
// enable unclean leader election globally, but disable for our specific test topic
configProps1.put("unclean.leader.election.enable", "true")
configProps2.put("unclean.leader.election.enable", "true")
@ -176,9 +176,9 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
verifyUncleanLeaderElectionDisabled()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testUncleanLeaderElectionInvalidTopicOverride(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testUncleanLeaderElectionInvalidTopicOverride(quorum: String, groupProtocol: String): Unit = {
startBrokers(Seq(configProps1))
// create topic with an invalid value for unclean leader election
@ -311,6 +311,7 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
val brokerList = TestUtils.plaintextBootstrapServers(brokers)
// Don't rely on coordinator as it may be down when this method is called
val consumer = TestUtils.createConsumer(brokerList,
groupProtocolFromTestParameters(),
groupId = "group" + random.nextLong(),
enableAutoCommit = false,
valueDeserializer = new StringDeserializer)
@ -322,9 +323,9 @@ class UncleanLeaderElectionTest extends QuorumTestHarness {
} finally consumer.close()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testTopicUncleanLeaderElectionEnableWithAlterTopicConfigs(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testTopicUncleanLeaderElectionEnableWithAlterTopicConfigs(quorum: String, groupProtocol: String): Unit = {
// unclean leader election is disabled by default
startBrokers(Seq(configProps1, configProps2))

View File

@ -25,6 +25,7 @@ import org.junit.jupiter.api.Assertions._
import kafka.integration.KafkaServerTestHarness
import kafka.server._
import kafka.utils._
import org.apache.kafka.clients.consumer.GroupProtocol
import scala.collection._
import scala.jdk.CollectionConverters._
@ -38,7 +39,7 @@ import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, L
import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics
import org.junit.jupiter.api.Timeout
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
@Timeout(120)
class MetricsTest extends KafkaServerTestHarness with Logging {
@ -57,7 +58,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
val nMessages = 2
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testMetricsReporterAfterDeletingTopic(quorum: String): Unit = {
val topic = "test-topic-metric"
createTopic(topic)
@ -67,7 +68,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testBrokerTopicMetricsUnregisteredAfterDeletingTopic(quorum: String): Unit = {
val topic = "test-broker-topic-metric"
createTopic(topic, 2)
@ -82,7 +83,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testClusterIdMetric(quorum: String): Unit = {
// Check if clusterId metric exists.
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
@ -90,7 +91,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testBrokerStateMetric(quorum: String): Unit = {
// Check if BrokerState metric exists.
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
@ -98,7 +99,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testYammerMetricsCountMetric(quorum: String): Unit = {
// Check if yammer-metrics-count metric exists.
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
@ -106,7 +107,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testLinuxIoMetrics(quorum: String): Unit = {
// Check if linux-disk-{read,write}-bytes metrics either do or do not exist depending on whether we are or are not
// able to collect those metrics on the platform where this test is running.
@ -118,7 +119,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testJMXFilter(quorum: String): Unit = {
// Check if cluster id metrics is not exposed in JMX
assertTrue(ManagementFactory.getPlatformMBeanServer
@ -128,7 +129,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUpdateJMXFilter(quorum: String): Unit = {
// verify previously exposed metrics are removed and existing matching metrics are added
brokers.foreach(broker => broker.kafkaYammerMetrics.reconfigure(
@ -141,7 +142,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testGeneralBrokerTopicMetricsAreGreedilyRegistered(quorum: String): Unit = {
val topic = "test-broker-topic-metric"
createTopic(topic, 2)
@ -157,7 +158,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testWindowsStyleTagNames(quorum: String): Unit = {
val path = "C:\\windows-path\\kafka-logs"
val tags = Map("dir" -> path)
@ -166,9 +167,9 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
assert(metric.getMBeanName.endsWith(expectedMBeanName))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testBrokerTopicMetricsBytesInOut(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testBrokerTopicMetricsBytesInOut(quorum: String, groupProtocol: String): Unit = {
val topic = "test-bytes-in-out"
val replicationBytesIn = BrokerTopicMetrics.REPLICATION_BYTES_IN_PER_SEC
val replicationBytesOut = BrokerTopicMetrics.REPLICATION_BYTES_OUT_PER_SEC
@ -192,7 +193,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
}
// Consume messages to make bytesOut tick
TestUtils.consumeTopicRecords(brokers, topic, nMessages)
TestUtils.consumeTopicRecords(brokers, topic, nMessages, GroupProtocol.of(groupProtocol))
val initialReplicationBytesIn = TestUtils.meterCount(replicationBytesIn)
val initialReplicationBytesOut = TestUtils.meterCount(replicationBytesOut)
val initialBytesIn = TestUtils.meterCount(bytesIn)
@ -208,7 +209,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
assertEquals(initialBytesOut, TestUtils.meterCount(bytesOut))
// Consume messages to make bytesOut tick
TestUtils.consumeTopicRecords(brokers, topic, nMessages)
TestUtils.consumeTopicRecords(brokers, topic, nMessages, GroupProtocol.of(groupProtocol))
assertTrue(TestUtils.meterCount(bytesOut) > initialBytesOut)
}
@ -267,7 +268,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
* and testZooKeeperSessionStateMetric in ZooKeeperClientTest test the metrics behaviour.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testSessionExpireListenerMetrics(quorum: String): Unit = {
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
val expectedNumMetrics = if (isKRaftTest()) 0 else 1

View File

@ -206,9 +206,7 @@ object AddPartitionsToTxnRequestServerTest {
def parameters: JStream[Arguments] = {
val arguments = mutable.ListBuffer[Arguments]()
ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions().forEach { version =>
Array("kraft", "zk").foreach { quorum =>
arguments += Arguments.of(quorum, version)
}
arguments += Arguments.of("kraft", version)
}
arguments.asJava.stream()
}

View File

@ -39,7 +39,7 @@ class ApiVersionsResponseIntegrationTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testSendV3ApiVersionsRequest(quorum: String): Unit = {
val response = sendApiVersionsRequest(3)
if (quorum.equals("kraft")) {
@ -51,7 +51,7 @@ class ApiVersionsResponseIntegrationTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testSendV4ApiVersionsRequest(quorum: String): Unit = {
val response = sendApiVersionsRequest(4)
if (quorum.equals("kraft")) {

View File

@ -127,7 +127,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testSetUnsetQuota(quorum: String): Unit = {
val rate = 1.5
val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "User")
@ -144,7 +144,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testQuotaMetric(quorum: String): Unit = {
asPrincipal(ThrottledPrincipal) {
// Metric is lazily created
@ -167,7 +167,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testStrictCreateTopicsRequest(quorum: String): Unit = {
asPrincipal(ThrottledPrincipal) {
// Create two topics worth of 30 partitions each. As we use a strict quota, we
@ -191,7 +191,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testPermissiveCreateTopicsRequest(quorum: String): Unit = {
asPrincipal(ThrottledPrincipal) {
// Create two topics worth of 30 partitions each. As we use a permissive quota, we
@ -205,7 +205,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUnboundedCreateTopicsRequest(quorum: String): Unit = {
asPrincipal(UnboundedPrincipal) {
// Create two topics worth of 30 partitions each. As we use an user without quota, we
@ -217,7 +217,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testStrictDeleteTopicsRequest(quorum: String): Unit = {
asPrincipal(UnboundedPrincipal) {
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
@ -245,7 +245,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testPermissiveDeleteTopicsRequest(quorum: String): Unit = {
asPrincipal(UnboundedPrincipal) {
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
@ -263,7 +263,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUnboundedDeleteTopicsRequest(quorum: String): Unit = {
asPrincipal(UnboundedPrincipal) {
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
@ -277,7 +277,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testStrictCreatePartitionsRequest(quorum: String): Unit = {
asPrincipal(UnboundedPrincipal) {
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
@ -305,7 +305,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testPermissiveCreatePartitionsRequest(quorum: String): Unit = {
asPrincipal(UnboundedPrincipal) {
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
@ -323,7 +323,7 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUnboundedCreatePartitionsRequest(quorum: String): Unit = {
asPrincipal(UnboundedPrincipal) {
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)

View File

@ -33,7 +33,7 @@ import scala.jdk.CollectionConverters._
class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testValidCreateTopicsRequests(quorum: String): Unit = {
// Generated assignments
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1"))))
@ -63,7 +63,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testErrorCreateTopicsRequests(quorum: String): Unit = {
val existingTopic = "existing-topic"
createTopic(existingTopic)
@ -134,7 +134,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testInvalidCreateTopicsRequests(quorum: String): Unit = {
// Partitions/ReplicationFactor and ReplicaAssignment
validateErrorCreateTopicsRequests(topicsReq(Seq(
@ -201,7 +201,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateClusterMetadataTopic(quorum: String): Unit = {
validateErrorCreateTopicsRequests(
topicsReq(Seq(topicReq(Topic.CLUSTER_METADATA_TOPIC_NAME))),

View File

@ -47,7 +47,7 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testValidCreateTopicsRequests(quorum: String): Unit = {
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1",
numPartitions = 5))))
@ -66,7 +66,7 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testErrorCreateTopicsRequests(quorum: String): Unit = {
val existingTopic = "existing-topic"
createTopic(existingTopic, 5)

View File

@ -49,7 +49,7 @@ class DelegationTokenRequestsOnPlainTextTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("kraft", "zk"))
@ValueSource(strings = Array("kraft"))
def testDelegationTokenRequests(quorum: String): Unit = {
adminClient = Admin.create(createAdminConfig)

View File

@ -66,7 +66,7 @@ class DelegationTokenRequestsTest extends IntegrationTestHarness with SaslSetup
}
@ParameterizedTest
@ValueSource(strings = Array("kraft", "zk"))
@ValueSource(strings = Array("kraft"))
def testDelegationTokenRequests(quorum: String): Unit = {
adminClient = Admin.create(createAdminConfig)

View File

@ -56,7 +56,7 @@ class DelegationTokenRequestsWithDisableTokenFeatureTest extends BaseRequestTest
}
@ParameterizedTest
@ValueSource(strings = Array("kraft", "zk"))
@ValueSource(strings = Array("kraft"))
def testDelegationTokenRequests(quorum: String): Unit = {
adminClient = Admin.create(createAdminConfig)

View File

@ -37,7 +37,7 @@ class DeleteRecordsRequestTest extends BaseRequestTest {
private val MESSAGES_PRODUCED_PER_PARTITION = 10
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDeleteRecordsHappyCase(quorum: String): Unit = {
val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords
@ -62,7 +62,7 @@ class DeleteRecordsRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testErrorWhenDeletingRecordsWithInvalidOffset(quorum: String): Unit = {
val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords
@ -87,7 +87,7 @@ class DeleteRecordsRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testErrorWhenDeletingRecordsWithInvalidTopic(quorum: String): Unit = {
val invalidTopicPartition = new TopicPartition("invalid-topic", 0)
// Create the DeleteRecord request requesting deletion of offset which is not present

View File

@ -39,7 +39,7 @@ import scala.jdk.CollectionConverters._
class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testTopicDeletionClusterHasOfflinePartitions(quorum: String): Unit = {
// Create a two topics with one partition/replica. Make one of them offline.
val offlineTopic = "topic-1"
@ -73,7 +73,7 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testValidDeleteTopicRequests(quorum: String): Unit = {
val timeout = 10000
// Single topic

View File

@ -49,7 +49,7 @@ class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDeleteRecordsRequest(quorum: String): Unit = {
val topic = "topic-1"
val request = new DeleteTopicsRequest.Builder(

View File

@ -49,13 +49,13 @@ class DescribeClusterRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDescribeClusterRequestIncludingClusterAuthorizedOperations(quorum: String): Unit = {
testDescribeClusterRequest(true)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDescribeClusterRequestExcludingClusterAuthorizedOperations(quorum: String): Unit = {
testDescribeClusterRequest(false)
}

View File

@ -65,7 +65,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testConfigChange(quorum: String): Unit = {
if (!isKRaftTest()) {
assertTrue(this.servers.head.dynamicConfigHandlers.contains(ConfigType.TOPIC),
@ -109,7 +109,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDynamicTopicConfigChange(quorum: String): Unit = {
val tp = new TopicPartition("test", 0)
val oldSegmentSize = 1000
@ -230,7 +230,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testClientIdQuotaConfigChange(quorum: String): Unit = {
val m = new util.HashMap[String, String]
m.put(CLIENT_ID, "testClient")
@ -238,7 +238,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUserQuotaConfigChange(quorum: String): Unit = {
val m = new util.HashMap[String, String]
m.put(USER, "ANONYMOUS")
@ -246,7 +246,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testUserClientIdQuotaChange(quorum: String): Unit = {
val m = new util.HashMap[String, String]
m.put(USER, "ANONYMOUS")
@ -255,7 +255,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDefaultClientIdQuotaConfigChange(quorum: String): Unit = {
val m = new util.HashMap[String, String]
m.put(CLIENT_ID, null)
@ -263,7 +263,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDefaultUserQuotaConfigChange(quorum: String): Unit = {
val m = new util.HashMap[String, String]
m.put(USER, null)
@ -271,7 +271,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDefaultUserClientIdQuotaConfigChange(quorum: String): Unit = {
val m = new util.HashMap[String, String]
m.put(USER, null)
@ -312,7 +312,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testIpQuotaInitialization(quorum: String): Unit = {
val broker = brokers.head
if (isKRaftTest()) {
@ -352,7 +352,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testIpQuotaConfigChange(quorum: String): Unit = {
val admin = createAdminClient()
try {
@ -405,7 +405,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
private def tempTopic() : String = "testTopic" + random.nextInt(1000000)
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testConfigChangeOnNonExistingTopicWithAdminClient(quorum: String): Unit = {
val topic = tempTopic()
val admin = createAdminClient()
@ -456,7 +456,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testIncrementalAlterDefaultTopicConfig(quorum: String): Unit = {
val admin = createAdminClient()
try {
@ -471,7 +471,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
@nowarn("cat=deprecation")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testAlterDefaultTopicConfig(quorum: String): Unit = {
val admin = createAdminClient()
try {
@ -514,7 +514,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testBrokerIdConfigChangeAndDelete(quorum: String): Unit = {
val newValue: Long = 100000L
val brokerId: String = this.brokers.head.config.brokerId.toString
@ -538,7 +538,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDefaultBrokerIdConfigChangeAndDelete(quorum: String): Unit = {
val newValue: Long = 100000L
val brokerId: String = ""
@ -561,7 +561,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testDefaultAndBrokerIdConfigChange(quorum: String): Unit = {
val newValue: Long = 100000L
val brokerId: String = this.brokers.head.config.brokerId.toString

View File

@ -119,7 +119,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testProduceRequestWithNullClientId(quorum: String): Unit = {
val topic = "topic"
val topicPartition = new TopicPartition(topic, 0)
@ -165,25 +165,25 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testHeaderOnlyRequest(quorum: String): Unit = {
verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, 1))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testInvalidApiKeyRequest(quorum: String): Unit = {
verifyDisconnect(requestHeaderBytes(-1, 0))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testInvalidApiVersionRequest(quorum: String): Unit = {
verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, -1))
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testMalformedHeaderRequest(quorum: String): Unit = {
val serializedBytes = {
// Only send apiKey and apiVersion

View File

@ -151,7 +151,7 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
* fetch requests.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testV1FetchFromConsumer(quorum: String): Unit = {
testV1Fetch(isFollowerFetch = false)
}
@ -160,7 +160,7 @@ class FetchRequestDownConversionConfigTest extends BaseRequestTest {
* Tests that "message.downconversion.enable" has no effect on fetch requests from replicas.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testV1FetchFromReplica(quorum: String): Unit = {
testV1Fetch(isFollowerFetch = true)
}

View File

@ -105,7 +105,7 @@ class FetchRequestMaxBytesTest extends BaseRequestTest {
* returned in full even if this is larger than FetchMaxBytes. See KIP-74.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testConsumeMultipleRecords(quorum: String): Unit = {
createTopics()

View File

@ -43,7 +43,7 @@ import scala.util.Random
class FetchRequestTest extends BaseFetchRequestTest {
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testBrokerRespectsPartitionsOrderAndSizeLimits(quorum: String): Unit = {
initProducer()
@ -146,7 +146,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchRequestV4WithReadCommitted(quorum: String): Unit = {
initProducer()
val maxPartitionBytes = 200
@ -165,7 +165,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchRequestToNonReplica(quorum: String): Unit = {
val topic = "topic"
val partition = 0
@ -196,13 +196,13 @@ class FetchRequestTest extends BaseFetchRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testLastFetchedEpochValidation(quorum: String): Unit = {
checkLastFetchedEpochValidation(ApiKeys.FETCH.latestVersion())
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testLastFetchedEpochValidationV12(quorum: String): Unit = {
checkLastFetchedEpochValidation(12)
}
@ -251,13 +251,13 @@ class FetchRequestTest extends BaseFetchRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testCurrentEpochValidation(quorum: String): Unit = {
checkCurrentEpochValidation(ApiKeys.FETCH.latestVersion())
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testCurrentEpochValidationV12(quorum: String): Unit = {
checkCurrentEpochValidation(12)
}
@ -301,13 +301,13 @@ class FetchRequestTest extends BaseFetchRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testEpochValidationWithinFetchSession(quorum: String): Unit = {
checkEpochValidationWithinFetchSession(ApiKeys.FETCH.latestVersion())
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testEpochValidationWithinFetchSessionV12(quorum: String): Unit = {
checkEpochValidationWithinFetchSession(12)
}
@ -369,7 +369,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
* channel is muted in the server. If buffers are not released this will result in OOM.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testDownConversionWithConnectionFailure(quorum: String): Unit = {
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head
val topicIds = getTopicIds().asJava
@ -437,7 +437,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
* some records have to be dropped during the conversion.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testDownConversionFromBatchedToUnbatchedRespectsOffset(quorum: String): Unit = {
// Increase linger so that we have control over the batches created
producer = TestUtils.createProducer(bootstrapServers(),
@ -519,7 +519,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
* This tests using FetchRequests that don't use topic IDs
*/
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testCreateIncrementalFetchWithPartitionsInErrorV12(quorum: String): Unit = {
def createConsumerFetchRequest(topicPartitions: Seq[TopicPartition],
metadata: JFetchMetadata,
@ -582,7 +582,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
* Test that when a Fetch Request receives an unknown topic ID, it returns a top level error.
*/
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchWithPartitionsWithIdError(quorum: String): Unit = {
def createConsumerFetchRequest(fetchData: util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData],
metadata: JFetchMetadata,
@ -627,7 +627,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testZStdCompressedTopic(quorum: String): Unit = {
// ZSTD compressed topic
val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.ZSTD.name)
@ -675,7 +675,7 @@ class FetchRequestTest extends BaseFetchRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk","kraft"))
@ValueSource(strings = Array("kraft"))
def testZStdCompressedRecords(quorum: String): Unit = {
// Producer compressed topic
val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.PRODUCER.name)

View File

@ -64,7 +64,7 @@ class KafkaMetricReporterExceptionHandlingTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testBothReportersAreInvoked(quorum: String): Unit = {
val port = anySocketServer.boundPort(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))
val socket = new Socket("localhost", port)

View File

@ -81,7 +81,7 @@ class KafkaMetricsReporterTest extends QuorumTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testMetricsContextNamespacePresent(quorum: String): Unit = {
assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID.get())
if (isKRaftTest()) {

View File

@ -44,7 +44,7 @@ class ListOffsetsRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testListOffsetsErrorCodes(quorum: String): Unit = {
val targetTimes = List(new ListOffsetsTopic()
.setName(topic)
@ -109,7 +109,7 @@ class ListOffsetsRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCurrentEpochValidation(quorum: String): Unit = {
val topic = "topic"
val topicPartition = new TopicPartition(topic, 0)
@ -176,7 +176,7 @@ class ListOffsetsRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testResponseIncludesLeaderEpoch(quorum: String): Unit = {
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
val firstLeaderId = partitionToLeader(partition.partition)
@ -217,7 +217,7 @@ class ListOffsetsRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testResponseDefaultOffsetAndLeaderEpochForAllVersions(quorum: String): Unit = {
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
val firstLeaderId = partitionToLeader(partition.partition)

View File

@ -22,7 +22,7 @@ import java.util.concurrent.{ExecutionException, TimeUnit}
import kafka.api.IntegrationTestHarness
import kafka.controller.{OfflineReplica, PartitionAndReplica}
import kafka.utils.TestUtils.{Checkpoint, LogDirFailureType, Roll, waitUntilTrue}
import kafka.utils.{CoreUtils, TestUtils}
import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils}
import org.apache.kafka.clients.consumer.Consumer
import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
import org.apache.kafka.common.TopicPartition
@ -32,7 +32,7 @@ import org.apache.kafka.metadata.BrokerState
import org.apache.kafka.server.config.{ReplicationConfigs, ServerConfigs, ServerLogConfigs}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.MethodSource
import org.junit.jupiter.params.ParameterizedTest
import java.nio.file.Files
@ -63,15 +63,15 @@ class LogDirFailureTest extends IntegrationTestHarness {
ensureConsistentKRaftMetadata()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testProduceErrorFromFailureOnLogRoll(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testProduceErrorFromFailureOnLogRoll(quorum: String, groupProtocol: String): Unit = {
testProduceErrorsFromLogDirFailureOnLeader(Roll)
}
@ParameterizedTest
@ValueSource(strings = Array("kraft"))
def testLogDirNotificationTimeout(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testLogDirNotificationTimeout(quorum: String, groupProtocol: String): Unit = {
// Disable retries to allow exception to bubble up for validation
this.producerConfig.setProperty(ProducerConfig.RETRIES_CONFIG, "0")
this.producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false")
@ -94,9 +94,9 @@ class LogDirFailureTest extends IntegrationTestHarness {
leaderServer.awaitShutdown()
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testIOExceptionDuringLogRoll(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testIOExceptionDuringLogRoll(quorum: String, groupProtocol: String): Unit = {
testProduceAfterLogDirFailureOnLeader(Roll, quorum)
}
@ -131,21 +131,21 @@ class LogDirFailureTest extends IntegrationTestHarness {
}
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testProduceErrorFromFailureOnCheckpoint(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testProduceErrorFromFailureOnCheckpoint(quorum: String, groupProtocol: String): Unit = {
testProduceErrorsFromLogDirFailureOnLeader(Checkpoint)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testIOExceptionDuringCheckpoint(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testIOExceptionDuringCheckpoint(quorum: String, groupProtocol: String): Unit = {
testProduceAfterLogDirFailureOnLeader(Checkpoint, quorum)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testReplicaFetcherThreadAfterLogDirFailureOnFollower(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testReplicaFetcherThreadAfterLogDirFailureOnFollower(quorum: String, groupProtocol: String): Unit = {
this.producerConfig.setProperty(ProducerConfig.RETRIES_CONFIG, "0")
this.producerConfig.setProperty(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, "false")
val producer = createProducer()

View File

@ -57,7 +57,7 @@ class LogOffsetTest extends BaseRequestTest {
@deprecated("ListOffsetsRequest V0", since = "")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testGetOffsetsForUnknownTopic(quorum: String): Unit = {
val topicPartition = new TopicPartition("foo", 0)
val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
@ -68,7 +68,7 @@ class LogOffsetTest extends BaseRequestTest {
@deprecated("ListOffsetsRequest V0", since = "")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testGetOffsetsAfterDeleteRecords(quorum: String): Unit = {
val topic = "kafka-"
val topicPartition = new TopicPartition(topic, 0)
@ -94,7 +94,7 @@ class LogOffsetTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchOffsetByTimestampForMaxTimestampAfterTruncate(quorum: String): Unit = {
val topic = "kafka-"
val topicPartition = new TopicPartition(topic, 0)
@ -116,7 +116,7 @@ class LogOffsetTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchOffsetByTimestampForMaxTimestampWithUnorderedTimestamps(quorum: String): Unit = {
val topic = "kafka-"
val topicPartition = new TopicPartition(topic, 0)
@ -135,7 +135,7 @@ class LogOffsetTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testGetOffsetsBeforeLatestTime(quorum: String): Unit = {
val topic = "kafka-"
val topicPartition = new TopicPartition(topic, 0)
@ -168,7 +168,7 @@ class LogOffsetTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testEmptyLogsGetOffsets(quorum: String): Unit = {
val random = new Random
val topic = "kafka-"
@ -192,7 +192,7 @@ class LogOffsetTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchOffsetByTimestampForMaxTimestampWithEmptyLog(quorum: String): Unit = {
val topic = "kafka-"
val topicPartition = new TopicPartition(topic, 0)
@ -206,7 +206,7 @@ class LogOffsetTest extends BaseRequestTest {
@deprecated("legacyFetchOffsetsBefore", since = "")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testGetOffsetsBeforeNow(quorum: String): Unit = {
val random = new Random
val topic = "kafka-"
@ -236,7 +236,7 @@ class LogOffsetTest extends BaseRequestTest {
@deprecated("legacyFetchOffsetsBefore", since = "")
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testGetOffsetsBeforeEarliestTime(quorum: String): Unit = {
val random = new Random
val topic = "kafka-"
@ -264,7 +264,7 @@ class LogOffsetTest extends BaseRequestTest {
/* We test that `fetchOffsetsBefore` works correctly if `LogSegment.size` changes after each invocation (simulating
* a race condition) */
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchOffsetsBeforeWithChangingSegmentSize(quorum: String): Unit = {
val log: UnifiedLog = mock(classOf[UnifiedLog])
val logSegment: LogSegment = mock(classOf[LogSegment])
@ -280,7 +280,7 @@ class LogOffsetTest extends BaseRequestTest {
/* We test that `fetchOffsetsBefore` works correctly if `Log.logSegments` content and size are
* different (simulating a race condition) */
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testFetchOffsetsBeforeWithChangingSegments(quorum: String): Unit = {
val log: UnifiedLog = mock(classOf[UnifiedLog])
val logSegment: LogSegment = mock(classOf[LogSegment])

View File

@ -105,7 +105,7 @@ class LogRecoveryTest extends QuorumTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testHWCheckpointNoFailuresSingleLogSegment(quorum: String): Unit = {
val numMessages = 2L
sendMessages(numMessages.toInt)
@ -123,7 +123,7 @@ class LogRecoveryTest extends QuorumTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testHWCheckpointWithFailuresSingleLogSegment(quorum: String): Unit = {
var leader = getLeaderIdForPartition(servers, topicPartition)
@ -184,7 +184,7 @@ class LogRecoveryTest extends QuorumTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testHWCheckpointNoFailuresMultipleLogSegments(quorum: String): Unit = {
sendMessages(20)
val hw = 20L
@ -201,7 +201,7 @@ class LogRecoveryTest extends QuorumTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testHWCheckpointWithFailuresMultipleLogSegments(quorum: String): Unit = {
var leader = getLeaderIdForPartition(servers, topicPartition)

View File

@ -42,7 +42,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testClusterIdWithRequestVersion1(quorum: String): Unit = {
val v1MetadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort))
val v1ClusterId = v1MetadataResponse.clusterId
@ -50,7 +50,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testClusterIdIsValid(quorum: String): Unit = {
val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(2.toShort))
isValidClusterId(metadataResponse.clusterId)
@ -84,7 +84,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testRack(quorum: String): Unit = {
val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort))
// Validate rack matches what's set in generateConfigs() above
@ -94,7 +94,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testIsInternal(quorum: String): Unit = {
val internalTopic = Topic.GROUP_METADATA_TOPIC_NAME
val notInternalTopic = "notInternal"
@ -116,7 +116,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testNoTopicsRequest(quorum: String): Unit = {
// create some topics
createTopic("t1", 3, 2)
@ -130,7 +130,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testAutoTopicCreation(quorum: String): Unit = {
val topic1 = "t1"
val topic2 = "t2"
@ -161,7 +161,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testAutoCreateTopicWithInvalidReplicationFactor(quorum: String): Unit = {
// Shutdown all but one broker so that the number of brokers is less than the default replication factor
brokers.tail.foreach(_.shutdown())
@ -212,7 +212,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testAllTopicsRequest(quorum: String): Unit = {
// create some topics
createTopic("t1", 3, 2)
@ -230,7 +230,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testTopicIdsInResponse(quorum: String): Unit = {
val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1))
val topic1 = "topic1"
@ -260,7 +260,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
* Preferred replica should be the first item in the replicas list
*/
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testPreferredReplica(quorum: String): Unit = {
val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1))
createTopicWithAssignment("t1", replicaAssignment)
@ -284,7 +284,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testReplicaDownResponse(quorum: String): Unit = {
val replicaDownTopic = "replicaDown"
val replicaCount = 3
@ -330,7 +330,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testIsrAfterBrokerShutDownAndJoinsBack(quorum: String): Unit = {
def checkIsr[B <: KafkaBroker](
brokers: Seq[B],
@ -368,7 +368,7 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testAliveBrokersWithNoTopics(quorum: String): Unit = {
def checkMetadata[B <: KafkaBroker](
brokers: Seq[B],

View File

@ -34,7 +34,7 @@ import scala.jdk.CollectionConverters._
class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testOffsetsForLeaderEpochErrorCodes(quorum: String): Unit = {
val topic = "topic"
val partition = new TopicPartition(topic, 0)
@ -59,7 +59,7 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCurrentEpochValidation(quorum: String): Unit = {
val topic = "topic"
val topicPartition = new TopicPartition(topic, 0)

View File

@ -47,7 +47,7 @@ class ProduceRequestTest extends BaseRequestTest {
val metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testSimpleProduceRequest(quorum: String): Unit = {
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
@ -132,7 +132,7 @@ class ProduceRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testProduceToNonReplica(quorum: String): Unit = {
val topic = "topic"
val partition = 0
@ -175,7 +175,7 @@ class ProduceRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCorruptLz4ProduceRequest(quorum: String): Unit = {
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
val timestamp = 1000000
@ -210,7 +210,7 @@ class ProduceRequestTest extends BaseRequestTest {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testZSTDProduceRequest(quorum: String): Unit = {
val topic = "topic"
val partition = 0

View File

@ -40,7 +40,7 @@ class ReplicaFetchTest extends IntegrationTestHarness {
override def brokerCount: Int = 2
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testReplicaFetcherThread(quorum: String): Unit = {
val partition = 0
val testMessageList1 = List("test1", "test2", "test3", "test4")

View File

@ -69,13 +69,13 @@ class ReplicationQuotasTest extends QuorumTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def shouldBootstrapTwoBrokersWithLeaderThrottle(quorum: String): Unit = {
shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(true)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def shouldBootstrapTwoBrokersWithFollowerThrottle(quorum: String): Unit = {
shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(false)
}
@ -206,7 +206,7 @@ class ReplicationQuotasTest extends QuorumTestHarness {
def tp(partition: Int): TopicPartition = new TopicPartition(topic, partition)
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def shouldThrottleOldSegments(quorum: String): Unit = {
/**
* Simple test which ensures throttled replication works when the dataset spans many segments

View File

@ -16,7 +16,7 @@
*/
package kafka.server
import kafka.utils.{CoreUtils, TestUtils}
import kafka.utils.{CoreUtils, TestInfoUtils, TestUtils}
import java.io.{DataInputStream, File}
import java.net.ServerSocket
@ -43,7 +43,7 @@ import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout}
import org.junit.jupiter.api.Assertions._
import org.junit.jupiter.api.function.Executable
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
import java.time.Duration
import java.util.Properties
@ -84,9 +84,9 @@ class ServerShutdownTest extends KafkaServerTestHarness {
super.setUp(testInfo)
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
def testCleanShutdown(quorum: String): Unit = {
@ParameterizedTest(name = TestInfoUtils.TestWithParameterizedQuorumAndGroupProtocolNames)
@MethodSource(Array("getTestQuorumAndGroupProtocolParametersAll"))
def testCleanShutdown(quorum: String, groupProtocol: String): Unit = {
def createProducer(): KafkaProducer[Integer, String] =
TestUtils.createProducer(
@ -98,6 +98,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
def createConsumer(): Consumer[Integer, String] =
TestUtils.createConsumer(
bootstrapServers(),
groupProtocolFromTestParameters(),
securityProtocol = SecurityProtocol.PLAINTEXT,
keyDeserializer = new IntegerDeserializer,
valueDeserializer = new StringDeserializer
@ -144,7 +145,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testCleanShutdownAfterFailedStartup(quorum: String): Unit = {
if (isKRaftTest()) {
propsToChangeUponRestart.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "1000")
@ -159,7 +160,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testNoCleanShutdownAfterFailedStartupDueToCorruptLogs(quorum: String): Unit = {
createTopic(topic)
shutdownBroker()
@ -245,7 +246,7 @@ class ServerShutdownTest extends KafkaServerTestHarness {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def testConsecutiveShutdown(quorum: String): Unit = {
shutdownBroker()
brokers.head.shutdown()

View File

@ -66,7 +66,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def shouldAddCurrentLeaderEpochToMessagesAsTheyAreWrittenToLeader(quorum: String): Unit = {
brokers ++= (0 to 1).map { id => createBroker(fromProps(createBrokerConfig(id, zkConnectOrNull))) }
@ -99,7 +99,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def shouldSendLeaderEpochRequestAndGetAResponse(quorum: String): Unit = {
//3 brokers, put partition on 100/101 and then pretend to be 102
@ -147,7 +147,7 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
}
@ParameterizedTest
@ValueSource(strings = Array("zk", "kraft"))
@ValueSource(strings = Array("kraft"))
def shouldIncreaseLeaderEpochBetweenLeaderRestarts(quorum: String): Unit = {
//Setup: we are only interested in the single partition on broker 101
brokers += createBroker(fromProps(createBrokerConfig(100, zkConnectOrNull)))

View File

@ -662,6 +662,7 @@ object TestUtils extends Logging {
* Create a consumer with a few pre-configured properties.
*/
def createConsumer[K, V](brokerList: String,
groupProtocol: GroupProtocol,
groupId: String = "group",
autoOffsetReset: String = "earliest",
enableAutoCommit: Boolean = true,
@ -671,8 +672,7 @@ object TestUtils extends Logging {
trustStoreFile: Option[File] = None,
saslProperties: Option[Properties] = None,
keyDeserializer: Deserializer[K] = new ByteArrayDeserializer,
valueDeserializer: Deserializer[V] = new ByteArrayDeserializer,
groupProtocol: GroupProtocol = GroupProtocol.CLASSIC): Consumer[K, V] = {
valueDeserializer: Deserializer[V] = new ByteArrayDeserializer): Consumer[K, V] = {
val consumerProps = new Properties
consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList)
consumerProps.put(ConsumerConfig.GROUP_PROTOCOL_CONFIG, groupProtocol.toString)
@ -1303,11 +1303,13 @@ object TestUtils extends Logging {
brokers: Seq[B],
topic: String,
numMessages: Int,
groupProtocol: GroupProtocol,
groupId: String = "group",
securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT,
trustStoreFile: Option[File] = None,
waitTime: Long = JTestUtils.DEFAULT_MAX_WAIT_MS): Seq[ConsumerRecord[Array[Byte], Array[Byte]]] = {
val consumer = createConsumer(bootstrapServers(brokers, ListenerName.forSecurityProtocol(securityProtocol)),
groupProtocol,
groupId = groupId,
securityProtocol = securityProtocol,
trustStoreFile = trustStoreFile)

View File

@ -35,7 +35,7 @@ import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.ArrayList;
import java.util.Collections;
@ -106,9 +106,9 @@ public abstract class TieredStorageTestHarness extends IntegrationTestHarness {
}
// NOTE: Not able to refer TestInfoUtils#TestWithParameterizedQuorumName() in the ParameterizedTest name.
@ParameterizedTest(name = "{displayName}.quorum={0}")
@ValueSource(strings = {"zk", "kraft"})
public void executeTieredStorageTest(String quorum) {
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}")
@MethodSource("getTestQuorumAndGroupProtocolParametersAll")
public void executeTieredStorageTest(String quorum, String groupProtocol) {
TieredStorageTestBuilder builder = new TieredStorageTestBuilder();
writeTestSpecifications(builder);
try {

View File

@ -22,7 +22,7 @@ import org.apache.kafka.tiered.storage.TieredStorageTestHarness;
import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.Arrays;
import java.util.Collections;
@ -41,10 +41,11 @@ public final class DisableRemoteLogOnTopicTest extends TieredStorageTestHarness
return 2;
}
@ParameterizedTest(name = "{displayName}.quorum={0}")
@ValueSource(strings = {"kraft"})
public void executeTieredStorageTest(String quorum) {
super.executeTieredStorageTest(quorum);
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}")
@MethodSource("getTestQuorumAndGroupProtocolParametersAll")
@Override
public void executeTieredStorageTest(String quorum, String groupProtocol) {
super.executeTieredStorageTest(quorum, groupProtocol);
}
@Override

View File

@ -25,7 +25,7 @@ import org.apache.kafka.tiered.storage.TieredStorageTestHarness;
import org.apache.kafka.tiered.storage.specs.KeyValueSpec;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import org.junit.jupiter.params.provider.MethodSource;
import java.util.Arrays;
import java.util.List;
@ -48,10 +48,11 @@ public class ListOffsetsTest extends TieredStorageTestHarness {
* 2. In Kraft mode, the leader-epoch gets bumped only for leader-election (0 -> 1) and not for reassignment.
* @param quorum The quorum to use for the test.
*/
@ParameterizedTest(name = "{displayName}.quorum={0}")
@ValueSource(strings = {"kraft"})
public void executeTieredStorageTest(String quorum) {
super.executeTieredStorageTest(quorum);
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}")
@MethodSource("getTestQuorumAndGroupProtocolParametersAll")
@Override
public void executeTieredStorageTest(String quorum, String groupProtocol) {
super.executeTieredStorageTest(quorum, groupProtocol);
}
@Override

View File

@ -30,9 +30,10 @@ import org.apache.kafka.test.TestUtils;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.function.Executable;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
import java.io.File;
import java.io.IOException;
@ -121,8 +122,10 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
closeSasl();
}
@Test
public void testConsumerGroupServiceWithAuthenticationFailure() throws Exception {
// NOTE: Not able to refer TestInfoUtils#TestWithParameterizedQuorumName() in the ParameterizedTest name.
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}")
@MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit")
public void testConsumerGroupServiceWithAuthenticationFailure(String quorum, String groupProtocol) throws Exception {
ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService();
try (Consumer<byte[], byte[]> consumer = createConsumer()) {
consumer.subscribe(Collections.singletonList(TOPIC));
@ -133,8 +136,9 @@ public class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
}
}
@Test
public void testConsumerGroupServiceWithAuthenticationSuccess() throws Exception {
@ParameterizedTest(name = "{displayName}.quorum={0}.groupProtocol={1}")
@MethodSource("getTestQuorumAndGroupProtocolParametersClassicGroupProtocolOnly_ZK_implicit")
public void testConsumerGroupServiceWithAuthenticationSuccess(String quorum, String groupProtocol) throws Exception {
createScramCredentialsViaPrivilegedAdminClient(JaasTestUtils.KAFKA_SCRAM_USER_2, JaasTestUtils.KAFKA_SCRAM_PASSWORD_2);
ConsumerGroupCommand.ConsumerGroupService consumerGroupService = prepareConsumerGroupService();
try (Consumer<byte[], byte[]> consumer = createConsumer()) {