mirror of https://github.com/apache/kafka.git
KAFKA-18695 Remove quorum=kraft and kip932 from all integration tests (#19633)
CI / build (push) Waiting to run
Details
CI / build (push) Waiting to run
Details
Currently, the quorum uses kraft by default, so there's no need to specify it explicitly. For kip932 and isShareGroupTest, they are no longer used after #19542 . Reviewers: PoAn Yang <payang@apache.org>, Ken Huang <s7133700@gmail.com>, Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
parent
54fd1361e5
commit
57ae6d6706
|
@ -50,9 +50,8 @@ import org.apache.kafka.storage.internals.checkpoint.CleanShutdownFileHandler;
|
||||||
|
|
||||||
import org.junit.jupiter.api.AfterEach;
|
import org.junit.jupiter.api.AfterEach;
|
||||||
import org.junit.jupiter.api.BeforeEach;
|
import org.junit.jupiter.api.BeforeEach;
|
||||||
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.api.TestInfo;
|
import org.junit.jupiter.api.TestInfo;
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource;
|
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.time.Duration;
|
import java.time.Duration;
|
||||||
|
@ -137,9 +136,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes
|
||||||
if (adminClient != null) adminClient.close();
|
if (adminClient != null) adminClient.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = {"kraft"})
|
public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr() throws ExecutionException, InterruptedException {
|
||||||
public void testHighWatermarkShouldNotAdvanceIfUnderMinIsr(String quorum) throws ExecutionException, InterruptedException {
|
|
||||||
adminClient.createTopics(
|
adminClient.createTopics(
|
||||||
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
||||||
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
||||||
|
@ -224,9 +222,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = {"kraft"})
|
public void testElrMemberCanBeElected() throws ExecutionException, InterruptedException {
|
||||||
public void testElrMemberCanBeElected(String quorum) throws ExecutionException, InterruptedException {
|
|
||||||
adminClient.createTopics(
|
adminClient.createTopics(
|
||||||
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
||||||
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
||||||
|
@ -300,9 +297,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = {"kraft"})
|
public void testElrMemberShouldBeKickOutWhenUncleanShutdown() throws ExecutionException, InterruptedException {
|
||||||
public void testElrMemberShouldBeKickOutWhenUncleanShutdown(String quorum) throws ExecutionException, InterruptedException {
|
|
||||||
adminClient.createTopics(
|
adminClient.createTopics(
|
||||||
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
||||||
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
||||||
|
@ -361,9 +357,8 @@ public class EligibleLeaderReplicasIntegrationTest extends KafkaServerTestHarnes
|
||||||
/*
|
/*
|
||||||
This test is only valid for KIP-966 part 1. When the unclean recovery is implemented, it should be removed.
|
This test is only valid for KIP-966 part 1. When the unclean recovery is implemented, it should be removed.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = {"kraft"})
|
public void testLastKnownLeaderShouldBeElectedIfEmptyElr() throws ExecutionException, InterruptedException {
|
||||||
public void testLastKnownLeaderShouldBeElectedIfEmptyElr(String quorum) throws ExecutionException, InterruptedException {
|
|
||||||
adminClient.createTopics(
|
adminClient.createTopics(
|
||||||
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
List.of(new NewTopic(testTopicName, 1, (short) 4))).all().get();
|
||||||
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
TestUtils.waitForPartitionMetadata(brokers(), testTopicName, 0, 1000);
|
||||||
|
|
|
@ -28,9 +28,9 @@ import org.apache.kafka.server.config.ServerLogConfigs
|
||||||
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentState}
|
import org.apache.kafka.server.log.remote.storage.{NoOpRemoteLogMetadataManager, NoOpRemoteStorageManager, RemoteLogManagerConfig, RemoteLogSegmentId, RemoteLogSegmentMetadata, RemoteLogSegmentState}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.function.Executable
|
import org.junit.jupiter.api.function.Executable
|
||||||
import org.junit.jupiter.api.{BeforeEach, Tag, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Tag, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
|
import org.junit.jupiter.params.provider.CsvSource
|
||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
@ -73,9 +73,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
testTopicName = s"${info.getTestMethod.get().getName}-${Random.alphanumeric.take(10).mkString}"
|
testTopicName = s"${info.getTestMethod.get().getName}-${Random.alphanumeric.take(10).mkString}"
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateRemoteTopicWithValidRetentionTime(): Unit = {
|
||||||
def testCreateRemoteTopicWithValidRetentionTime(quorum: String): Unit = {
|
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200")
|
topicConfig.put(TopicConfig.RETENTION_MS_CONFIG, "200")
|
||||||
|
@ -85,9 +84,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(topicConfig)
|
verifyRemoteLogTopicConfigs(topicConfig)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateRemoteTopicWithValidRetentionSize(): Unit = {
|
||||||
def testCreateRemoteTopicWithValidRetentionSize(quorum: String): Unit = {
|
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512")
|
topicConfig.put(TopicConfig.RETENTION_BYTES_CONFIG, "512")
|
||||||
|
@ -97,9 +95,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(topicConfig)
|
verifyRemoteLogTopicConfigs(topicConfig)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateRemoteTopicWithInheritedLocalRetentionTime(): Unit = {
|
||||||
def testCreateRemoteTopicWithInheritedLocalRetentionTime(quorum: String): Unit = {
|
|
||||||
// inherited local retention ms is 1000
|
// inherited local retention ms is 1000
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -109,9 +106,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(topicConfig)
|
verifyRemoteLogTopicConfigs(topicConfig)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateRemoteTopicWithInheritedLocalRetentionSize(): Unit = {
|
||||||
def testCreateRemoteTopicWithInheritedLocalRetentionSize(quorum: String): Unit = {
|
|
||||||
// inherited local retention bytes is 1024
|
// inherited local retention bytes is 1024
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -121,9 +117,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(topicConfig)
|
verifyRemoteLogTopicConfigs(topicConfig)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateRemoteTopicWithInvalidRetentionTime(): Unit = {
|
||||||
def testCreateRemoteTopicWithInvalidRetentionTime(quorum: String): Unit = {
|
|
||||||
// inherited local retention ms is 1000
|
// inherited local retention ms is 1000
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -133,9 +128,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
topicConfig = topicConfig))
|
topicConfig = topicConfig))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateRemoteTopicWithInvalidRetentionSize(): Unit = {
|
||||||
def testCreateRemoteTopicWithInvalidRetentionSize(quorum: String): Unit = {
|
|
||||||
// inherited local retention bytes is 1024
|
// inherited local retention bytes is 1024
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -145,9 +139,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
topicConfig = topicConfig))
|
topicConfig = topicConfig))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateCompactedRemoteStorage(): Unit = {
|
||||||
def testCreateCompactedRemoteStorage(quorum: String): Unit = {
|
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact")
|
topicConfig.put(TopicConfig.CLEANUP_POLICY_CONFIG, "compact")
|
||||||
|
@ -158,8 +151,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
|
|
||||||
// `remote.log.delete.on.disable` and `remote.log.copy.disable` only works in KRaft mode.
|
// `remote.log.delete.on.disable` and `remote.log.copy.disable` only works in KRaft mode.
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@CsvSource(Array("kraft,true,true", "kraft,true,false", "kraft,false,true", "kraft,false,false"))
|
@CsvSource(Array("true,true", "true,false", "false,true", "false,false"))
|
||||||
def testCreateRemoteTopicWithCopyDisabledAndDeleteOnDisable(quorum: String, copyDisabled: Boolean, deleteOnDisable: Boolean): Unit = {
|
def testCreateRemoteTopicWithCopyDisabledAndDeleteOnDisable(copyDisabled: Boolean, deleteOnDisable: Boolean): Unit = {
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, copyDisabled.toString)
|
topicConfig.put(TopicConfig.REMOTE_LOG_COPY_DISABLE_CONFIG, copyDisabled.toString)
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString)
|
topicConfig.put(TopicConfig.REMOTE_LOG_DELETE_ON_DISABLE_CONFIG, deleteOnDisable.toString)
|
||||||
|
@ -169,9 +162,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
}
|
}
|
||||||
|
|
||||||
// `remote.log.delete.on.disable` only works in KRaft mode.
|
// `remote.log.delete.on.disable` only works in KRaft mode.
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateTopicRetentionMsValidationWithRemoteCopyDisabled(): Unit = {
|
||||||
def testCreateTopicRetentionMsValidationWithRemoteCopyDisabled(quorum: String): Unit = {
|
|
||||||
val testTopicName2 = testTopicName + "2"
|
val testTopicName2 = testTopicName + "2"
|
||||||
val testTopicName3 = testTopicName + "3"
|
val testTopicName3 = testTopicName + "3"
|
||||||
val errorMsgMs = "When `remote.log.copy.disable` is set to true, the `local.retention.ms` and `retention.ms` " +
|
val errorMsgMs = "When `remote.log.copy.disable` is set to true, the `local.retention.ms` and `retention.ms` " +
|
||||||
|
@ -235,9 +227,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
admin.incrementalAlterConfigs(configs).all().get()
|
admin.incrementalAlterConfigs(configs).all().get()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateTopicRetentionBytesValidationWithRemoteCopyDisabled(): Unit = {
|
||||||
def testCreateTopicRetentionBytesValidationWithRemoteCopyDisabled(quorum: String): Unit = {
|
|
||||||
val testTopicName2 = testTopicName + "2"
|
val testTopicName2 = testTopicName + "2"
|
||||||
val testTopicName3 = testTopicName + "3"
|
val testTopicName3 = testTopicName + "3"
|
||||||
val errorMsgBytes = "When `remote.log.copy.disable` is set to true, the `local.retention.bytes` and `retention.bytes` " +
|
val errorMsgBytes = "When `remote.log.copy.disable` is set to true, the `local.retention.bytes` and `retention.bytes` " +
|
||||||
|
@ -300,9 +291,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
admin.incrementalAlterConfigs(configs).all().get()
|
admin.incrementalAlterConfigs(configs).all().get()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testEnableRemoteLogOnExistingTopicTest(): Unit = {
|
||||||
def testEnableRemoteLogOnExistingTopicTest(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor,
|
TestUtils.createTopicWithAdmin(admin, testTopicName, brokers, controllerServers, numPartitions, numReplicationFactor,
|
||||||
|
@ -318,9 +308,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(topicConfig)
|
verifyRemoteLogTopicConfigs(topicConfig)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testEnableRemoteLogWhenSystemRemoteStorageIsDisabled(): Unit = {
|
||||||
def testEnableRemoteLogWhenSystemRemoteStorageIsDisabled(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
|
|
||||||
val topicConfigWithRemoteStorage = new Properties()
|
val topicConfigWithRemoteStorage = new Properties()
|
||||||
|
@ -342,9 +331,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
assertTrue(errorMessage.getMessage.contains("Tiered Storage functionality is disabled in the broker"))
|
assertTrue(errorMessage.getMessage.contains("Tiered Storage functionality is disabled in the broker"))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUpdateTopicConfigWithValidRetentionTimeTest(): Unit = {
|
||||||
def testUpdateTopicConfigWithValidRetentionTimeTest(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -363,9 +351,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(topicConfig)
|
verifyRemoteLogTopicConfigs(topicConfig)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUpdateTopicConfigWithValidRetentionSizeTest(): Unit = {
|
||||||
def testUpdateTopicConfigWithValidRetentionSizeTest(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -384,9 +371,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(topicConfig)
|
verifyRemoteLogTopicConfigs(topicConfig)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUpdateTopicConfigWithInheritedLocalRetentionTime(): Unit = {
|
||||||
def testUpdateTopicConfigWithInheritedLocalRetentionTime(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -404,9 +390,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
() => admin.incrementalAlterConfigs(configs).all().get())
|
() => admin.incrementalAlterConfigs(configs).all().get())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUpdateTopicConfigWithInheritedLocalRetentionSize(): Unit = {
|
||||||
def testUpdateTopicConfigWithInheritedLocalRetentionSize(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.put(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -425,9 +410,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
}
|
}
|
||||||
|
|
||||||
// The remote storage config validation on controller level only works in KRaft
|
// The remote storage config validation on controller level only works in KRaft
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUpdateTopicConfigWithDisablingRemoteStorage(): Unit = {
|
||||||
def testUpdateTopicConfigWithDisablingRemoteStorage(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
val topicConfig = new Properties
|
val topicConfig = new Properties
|
||||||
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -446,9 +430,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
"If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.")
|
"If you want to disable remote storage and delete all remote data, please set `remote.storage.enable=false,remote.log.delete.on.disable=true`.")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUpdateTopicConfigWithDisablingRemoteStorageWithDeleteOnDisable(): Unit = {
|
||||||
def testUpdateTopicConfigWithDisablingRemoteStorageWithDeleteOnDisable(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
val topicConfig = new Properties
|
val topicConfig = new Properties
|
||||||
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
@ -473,9 +456,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
verifyRemoteLogTopicConfigs(newProps)
|
verifyRemoteLogTopicConfigs(newProps)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testTopicDeletion(): Unit = {
|
||||||
def testTopicDeletion(quorum: String): Unit = {
|
|
||||||
MyRemoteStorageManager.deleteSegmentEventCounter.set(0)
|
MyRemoteStorageManager.deleteSegmentEventCounter.set(0)
|
||||||
val numPartitions = 2
|
val numPartitions = 2
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
|
@ -492,9 +474,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
"Remote log segments should be deleted only once by the leader")
|
"Remote log segments should be deleted only once by the leader")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic(): Unit = {
|
||||||
def testClusterWideDisablementOfTieredStorageWithEnabledTieredTopic(quorum: String): Unit = {
|
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, "true")
|
||||||
|
|
||||||
|
@ -510,9 +491,8 @@ class RemoteTopicCrudTest extends IntegrationTestHarness {
|
||||||
faultHandler.setIgnore(true)
|
faultHandler.setIgnore(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled(): Unit = {
|
||||||
def testClusterWithoutTieredStorageStartsSuccessfullyIfTopicWithTieringDisabled(quorum: String): Unit = {
|
|
||||||
val topicConfig = new Properties()
|
val topicConfig = new Properties()
|
||||||
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, false.toString)
|
topicConfig.setProperty(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, false.toString)
|
||||||
|
|
||||||
|
|
|
@ -29,9 +29,7 @@ import org.apache.kafka.server.policy.AlterConfigPolicy
|
||||||
import org.apache.kafka.storage.internals.log.LogConfig
|
import org.apache.kafka.storage.internals.log.LogConfig
|
||||||
import org.apache.kafka.test.TestUtils.assertFutureThrows
|
import org.apache.kafka.test.TestUtils.assertFutureThrows
|
||||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue}
|
import org.junit.jupiter.api.Assertions.{assertEquals, assertNull, assertTrue}
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo, Timeout}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo, Timeout}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -79,9 +77,8 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
|
||||||
props.put(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[Policy])
|
props.put(ServerLogConfigs.ALTER_CONFIG_POLICY_CLASS_NAME_CONFIG, classOf[Policy])
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testValidAlterConfigs(): Unit = {
|
||||||
def testValidAlterConfigs(quorum: String): Unit = {
|
|
||||||
client = Admin.create(createConfig)
|
client = Admin.create(createConfig)
|
||||||
// Create topics
|
// Create topics
|
||||||
val topic1 = "describe-alter-configs-topic-1"
|
val topic1 = "describe-alter-configs-topic-1"
|
||||||
|
@ -100,16 +97,14 @@ class AdminClientWithPoliciesIntegrationTest extends KafkaServerTestHarness with
|
||||||
PlaintextAdminIntegrationTest.checkValidAlterConfigs(client, this, topicResource1, topicResource2, maxMessageBytes, retentionMs)
|
PlaintextAdminIntegrationTest.checkValidAlterConfigs(client, this, topicResource1, topicResource2, maxMessageBytes, retentionMs)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testInvalidAlterConfigs(): Unit = {
|
||||||
def testInvalidAlterConfigs(quorum: String): Unit = {
|
|
||||||
client = Admin.create(createConfig)
|
client = Admin.create(createConfig)
|
||||||
PlaintextAdminIntegrationTest.checkInvalidAlterConfigs(this, client)
|
PlaintextAdminIntegrationTest.checkInvalidAlterConfigs(this, client)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testInvalidAlterConfigsDueToPolicy(): Unit = {
|
||||||
def testInvalidAlterConfigsDueToPolicy(quorum: String): Unit = {
|
|
||||||
client = Admin.create(createConfig)
|
client = Admin.create(createConfig)
|
||||||
|
|
||||||
// Create topics
|
// Create topics
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -26,8 +26,7 @@ import org.apache.kafka.common.resource.ResourcePattern
|
||||||
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
import org.apache.kafka.common.security.auth.KafkaPrincipal
|
||||||
import org.apache.kafka.common.security.token.delegation.DelegationToken
|
import org.apache.kafka.common.security.token.delegation.DelegationToken
|
||||||
import org.junit.jupiter.api.Assertions.{assertThrows, assertTrue}
|
import org.junit.jupiter.api.Assertions.{assertThrows, assertTrue}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.Collections
|
import java.util.Collections
|
||||||
import scala.concurrent.ExecutionException
|
import scala.concurrent.ExecutionException
|
||||||
|
@ -94,18 +93,16 @@ class DelegationTokenEndToEndAuthorizationWithOwnerTest extends DelegationTokenE
|
||||||
createScramAdminClient(kafkaClientSaslMechanism, tokenRequesterPrincipal.getName, tokenRequesterPassword)
|
createScramAdminClient(kafkaClientSaslMechanism, tokenRequesterPrincipal.getName, tokenRequesterPassword)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateTokenForOtherUserFails(): Unit = {
|
||||||
def testCreateTokenForOtherUserFails(quorum: String): Unit = {
|
|
||||||
val thrown = assertThrows(classOf[ExecutionException], () => {
|
val thrown = assertThrows(classOf[ExecutionException], () => {
|
||||||
createDelegationTokens(() => new CreateDelegationTokenOptions().owner(otherClientPrincipal), assert = false)
|
createDelegationTokens(() => new CreateDelegationTokenOptions().owner(otherClientPrincipal), assert = false)
|
||||||
})
|
})
|
||||||
assertTrue(thrown.getMessage.contains("Delegation Token authorization failed"))
|
assertTrue(thrown.getMessage.contains("Delegation Token authorization failed"))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeTokenForOtherUserFails(): Unit = {
|
||||||
def testDescribeTokenForOtherUserFails(quorum: String): Unit = {
|
|
||||||
Using.resource(createScramAdminClient(kafkaClientSaslMechanism, describeTokenFailPrincipal.getName, describeTokenFailPassword)) { describeTokenFailAdminClient =>
|
Using.resource(createScramAdminClient(kafkaClientSaslMechanism, describeTokenFailPrincipal.getName, describeTokenFailPassword)) { describeTokenFailAdminClient =>
|
||||||
Using.resource(createScramAdminClient(kafkaClientSaslMechanism, otherClientPrincipal.getName, otherClientPassword)) { otherClientAdminClient =>
|
Using.resource(createScramAdminClient(kafkaClientSaslMechanism, otherClientPrincipal.getName, otherClientPassword)) { otherClientAdminClient =>
|
||||||
otherClientAdminClient.createDelegationToken().delegationToken().get()
|
otherClientAdminClient.createDelegationToken().delegationToken().get()
|
||||||
|
@ -117,9 +114,8 @@ class DelegationTokenEndToEndAuthorizationWithOwnerTest extends DelegationTokenE
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeTokenForOtherUserPasses(): Unit = {
|
||||||
def testDescribeTokenForOtherUserPasses(quorum: String): Unit = {
|
|
||||||
val adminClient = createTokenRequesterAdminClient()
|
val adminClient = createTokenRequesterAdminClient()
|
||||||
try {
|
try {
|
||||||
val tokens = adminClient.describeDelegationToken(
|
val tokens = adminClient.describeDelegationToken(
|
||||||
|
|
|
@ -79,9 +79,9 @@ class MetricsTest extends IntegrationTestHarness with SaslSetup {
|
||||||
/**
|
/**
|
||||||
* Verifies some of the metrics of producer, consumer as well as server.
|
* Verifies some of the metrics of producer, consumer as well as server.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest(name = "testMetrics with systemRemoteStorageEnabled: {1}")
|
@ParameterizedTest(name = "testMetrics with systemRemoteStorageEnabled: {0}")
|
||||||
@CsvSource(Array("kraft, true", "kraft, false"))
|
@CsvSource(Array("true", "false"))
|
||||||
def testMetrics(quorum: String, systemRemoteStorageEnabled: Boolean): Unit = {
|
def testMetrics(systemRemoteStorageEnabled: Boolean): Unit = {
|
||||||
val topic = "mytopic"
|
val topic = "mytopic"
|
||||||
createTopic(topic,
|
createTopic(topic,
|
||||||
numPartitions = 1,
|
numPartitions = 1,
|
||||||
|
|
|
@ -60,7 +60,7 @@ import org.apache.logging.log4j.core.config.Configurator
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo, Timeout}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo, Timeout}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
|
import org.junit.jupiter.params.provider.{MethodSource}
|
||||||
import org.slf4j.LoggerFactory
|
import org.slf4j.LoggerFactory
|
||||||
|
|
||||||
import java.util.AbstractMap.SimpleImmutableEntry
|
import java.util.AbstractMap.SimpleImmutableEntry
|
||||||
|
@ -2511,9 +2511,8 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kip932"))
|
def testListGroups(): Unit = {
|
||||||
def testListGroups(unused: String): Unit = {
|
|
||||||
val classicGroupId = "classic_group_id"
|
val classicGroupId = "classic_group_id"
|
||||||
val consumerGroupId = "consumer_group_id"
|
val consumerGroupId = "consumer_group_id"
|
||||||
val shareGroupId = "share_group_id"
|
val shareGroupId = "share_group_id"
|
||||||
|
@ -2643,9 +2642,7 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
def testShareGroups(): Unit = {
|
||||||
@ValueSource(strings = Array("kip932"))
|
|
||||||
def testShareGroups(unused: String): Unit = {
|
|
||||||
val testGroupId = "test_group_id"
|
val testGroupId = "test_group_id"
|
||||||
val testClientId = "test_client_id"
|
val testClientId = "test_client_id"
|
||||||
val fakeGroupId = "fake_group_id"
|
val fakeGroupId = "fake_group_id"
|
||||||
|
|
|
@ -21,10 +21,8 @@ import org.apache.kafka.common.network.ListenerName
|
||||||
import org.apache.kafka.common.security.auth._
|
import org.apache.kafka.common.security.auth._
|
||||||
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
|
import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuilder
|
||||||
import org.apache.kafka.clients.admin.AdminClientConfig
|
import org.apache.kafka.clients.admin.AdminClientConfig
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
import org.apache.kafka.common.errors.TopicAuthorizationException
|
import org.apache.kafka.common.errors.TopicAuthorizationException
|
||||||
|
|
||||||
// This test case uses a separate listener for client and inter-broker communication, from
|
// This test case uses a separate listener for client and inter-broker communication, from
|
||||||
|
@ -88,9 +86,8 @@ class PlaintextEndToEndAuthorizationTest extends EndToEndAuthorizationTest {
|
||||||
superuserClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers(interBrokerListenerName))
|
superuserClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers(interBrokerListenerName))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testListenerName(): Unit = {
|
||||||
def testListenerName(quorum: String): Unit = {
|
|
||||||
// To check the client listener name, establish a session on the server by sending any request eg sendRecords
|
// To check the client listener name, establish a session on the server by sending any request eg sendRecords
|
||||||
val producer = createProducer()
|
val producer = createProducer()
|
||||||
assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords = 1, tp))
|
assertThrows(classOf[TopicAuthorizationException], () => sendRecords(producer, numRecords = 1, tp))
|
||||||
|
|
|
@ -22,8 +22,7 @@ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord, Record
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
|
import org.apache.kafka.server.config.{ReplicationConfigs, ServerLogConfigs}
|
||||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals}
|
import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.nio.charset.StandardCharsets
|
import java.nio.charset.StandardCharsets
|
||||||
import java.util
|
import java.util
|
||||||
|
@ -50,9 +49,8 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
|
||||||
* Producer will attempt to send messages to the partition specified in each record, and should
|
* 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.
|
* succeed as long as the partition is included in the metadata.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testSendWithTopicDeletionMidWay(): Unit = {
|
||||||
def testSendWithTopicDeletionMidWay(quorum: String): Unit = {
|
|
||||||
val numRecords = 10
|
val numRecords = 10
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
|
|
||||||
|
@ -91,9 +89,8 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
|
||||||
* Producer will attempt to send messages to the partition specified in each record, and should
|
* Producer will attempt to send messages to the partition specified in each record, and should
|
||||||
* succeed as long as the metadata has been updated with new topic id.
|
* succeed as long as the metadata has been updated with new topic id.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testSendWithRecreatedTopic(): Unit = {
|
||||||
def testSendWithRecreatedTopic(quorum: String): Unit = {
|
|
||||||
val numRecords = 10
|
val numRecords = 10
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
createTopic(topic)
|
createTopic(topic)
|
||||||
|
@ -125,9 +122,8 @@ class ProducerSendWhileDeletionTest extends IntegrationTestHarness {
|
||||||
* Producer will attempt to send messages to the partition specified in each record, and should
|
* Producer will attempt to send messages to the partition specified in each record, and should
|
||||||
* succeed as long as the metadata cache on the leader includes the partition topic id.
|
* succeed as long as the metadata cache on the leader includes the partition topic id.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testSendWithTopicReassignmentIsMidWay(): Unit = {
|
||||||
def testSendWithTopicReassignmentIsMidWay(quorum: String): Unit = {
|
|
||||||
val numRecords = 10
|
val numRecords = 10
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val partition0: TopicPartition = new TopicPartition(topic, 0)
|
val partition0: TopicPartition = new TopicPartition(topic, 0)
|
||||||
|
|
|
@ -23,7 +23,7 @@ import org.apache.kafka.clients.consumer.{Consumer, ConsumerConfig}
|
||||||
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
|
||||||
import org.apache.kafka.common.{KafkaException, TopicPartition}
|
import org.apache.kafka.common.{KafkaException, TopicPartition}
|
||||||
import org.apache.kafka.common.errors.SaslAuthenticationException
|
import org.apache.kafka.common.errors.SaslAuthenticationException
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import kafka.utils.{TestInfoUtils, TestUtils}
|
import kafka.utils.{TestInfoUtils, TestUtils}
|
||||||
import org.apache.kafka.common.config.SaslConfigs
|
import org.apache.kafka.common.config.SaslConfigs
|
||||||
|
@ -91,7 +91,7 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
|
||||||
closeSasl()
|
closeSasl()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest(name="{displayName}.quorum=kraft.isIdempotenceEnabled={0}")
|
@ParameterizedTest(name="{displayName}.isIdempotenceEnabled={0}")
|
||||||
@ValueSource(booleans = Array(true, false))
|
@ValueSource(booleans = Array(true, false))
|
||||||
def testProducerWithAuthenticationFailure(isIdempotenceEnabled: Boolean): Unit = {
|
def testProducerWithAuthenticationFailure(isIdempotenceEnabled: Boolean): Unit = {
|
||||||
val prop = new Properties()
|
val prop = new Properties()
|
||||||
|
@ -111,9 +111,8 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
|
||||||
verifyWithRetry(sendOneRecord(producer2))()
|
verifyWithRetry(sendOneRecord(producer2))()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testTransactionalProducerWithAuthenticationFailure(): Unit = {
|
||||||
def testTransactionalProducerWithAuthenticationFailure(quorum: String): Unit = {
|
|
||||||
val txProducer = createTransactionalProducer()
|
val txProducer = createTransactionalProducer()
|
||||||
verifyAuthenticationException(txProducer.initTransactions())
|
verifyAuthenticationException(txProducer.initTransactions())
|
||||||
|
|
||||||
|
@ -157,9 +156,8 @@ class SaslClientsWithInvalidCredentialsTest extends AbstractSaslTest {
|
||||||
verifyWithRetry(consumer.poll(Duration.ofMillis(1000)))(_.count == 1)
|
verifyWithRetry(consumer.poll(Duration.ofMillis(1000)))(_.count == 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testKafkaAdminClientWithAuthenticationFailure(): Unit = {
|
||||||
def testKafkaAdminClientWithAuthenticationFailure(quorum: String): Unit = {
|
|
||||||
val props = JaasTestUtils.adminClientSecurityConfigs(securityProtocol, OptionConverters.toJava(trustStoreFile), OptionConverters.toJava(clientSaslProperties))
|
val props = JaasTestUtils.adminClientSecurityConfigs(securityProtocol, OptionConverters.toJava(trustStoreFile), OptionConverters.toJava(clientSaslProperties))
|
||||||
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
|
props.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers())
|
||||||
val adminClient = Admin.create(props)
|
val adminClient = Admin.create(props)
|
||||||
|
|
|
@ -34,9 +34,7 @@ import org.apache.kafka.common.{KafkaException, Uuid, requests}
|
||||||
import org.apache.kafka.network.SocketServerConfigs
|
import org.apache.kafka.network.SocketServerConfigs
|
||||||
import org.apache.kafka.server.config.QuotaConfig
|
import org.apache.kafka.server.config.QuotaConfig
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.io.IOException
|
import java.io.IOException
|
||||||
import java.net.{InetAddress, Socket}
|
import java.net.{InetAddress, Socket}
|
||||||
|
@ -84,9 +82,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Flaky("KAFKA-17999")
|
@Flaky("KAFKA-17999")
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDynamicConnectionQuota(): Unit = {
|
||||||
def testDynamicConnectionQuota(quorum: String): Unit = {
|
|
||||||
val maxConnectionsPerIP = 5
|
val maxConnectionsPerIP = 5
|
||||||
|
|
||||||
def connectAndVerify(): Unit = {
|
def connectAndVerify(): Unit = {
|
||||||
|
@ -112,9 +109,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
||||||
verifyMaxConnections(maxConnectionsPerIPOverride, connectAndVerify)
|
verifyMaxConnections(maxConnectionsPerIPOverride, connectAndVerify)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDynamicListenerConnectionQuota(): Unit = {
|
||||||
def testDynamicListenerConnectionQuota(quorum: String): Unit = {
|
|
||||||
val initialConnectionCount = connectionCount
|
val initialConnectionCount = connectionCount
|
||||||
|
|
||||||
def connectAndVerify(): Unit = {
|
def connectAndVerify(): Unit = {
|
||||||
|
@ -185,9 +181,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDynamicListenerConnectionCreationRateQuota(): Unit = {
|
||||||
def testDynamicListenerConnectionCreationRateQuota(quorum: String): Unit = {
|
|
||||||
// Create another listener. PLAINTEXT is an inter-broker listener
|
// Create another listener. PLAINTEXT is an inter-broker listener
|
||||||
// keep default limits
|
// keep default limits
|
||||||
val newListenerNames = Seq("PLAINTEXT", "EXTERNAL")
|
val newListenerNames = Seq("PLAINTEXT", "EXTERNAL")
|
||||||
|
@ -247,9 +242,8 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
|
||||||
waitForConnectionCount(initialConnectionCount)
|
waitForConnectionCount(initialConnectionCount)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDynamicIpConnectionRateQuota(): Unit = {
|
||||||
def testDynamicIpConnectionRateQuota(quorum: String): Unit = {
|
|
||||||
val connRateLimit = 10
|
val connRateLimit = 10
|
||||||
val initialConnectionCount = connectionCount
|
val initialConnectionCount = connectionCount
|
||||||
// before setting connection rate to 10, verify we can do at least double that by default (no limit)
|
// before setting connection rate to 10, verify we can do at least double that by default (no limit)
|
||||||
|
|
|
@ -24,9 +24,7 @@ import org.apache.kafka.common.network.ListenerName
|
||||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||||
import org.apache.kafka.network.SocketServerConfigs
|
import org.apache.kafka.network.SocketServerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -66,9 +64,8 @@ class DynamicNumNetworkThreadsTest extends BaseRequestTest {
|
||||||
.count(listener == _.tags().get("listener"))
|
.count(listener == _.tags().get("listener"))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDynamicNumNetworkThreads(): Unit = {
|
||||||
def testDynamicNumNetworkThreads(quorum: String): Unit = {
|
|
||||||
// Increase the base network thread count
|
// Increase the base network thread count
|
||||||
val newBaseNetworkThreadsCount = SocketServerConfigs.NUM_NETWORK_THREADS_DEFAULT + 1
|
val newBaseNetworkThreadsCount = SocketServerConfigs.NUM_NETWORK_THREADS_DEFAULT + 1
|
||||||
var props = new Properties
|
var props = new Properties
|
||||||
|
|
|
@ -38,9 +38,9 @@ import org.apache.kafka.common.security.kerberos.KerberosLogin
|
||||||
import org.apache.kafka.common.utils.{LogContext, MockTime}
|
import org.apache.kafka.common.utils.{LogContext, MockTime}
|
||||||
import org.apache.kafka.network.SocketServerConfigs
|
import org.apache.kafka.network.SocketServerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
|
import org.junit.jupiter.params.provider.MethodSource
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
|
@ -92,9 +92,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
|
||||||
* Tests that Kerberos replay error `Request is a replay (34)` is not handled as an authentication exception
|
* Tests that Kerberos replay error `Request is a replay (34)` is not handled as an authentication exception
|
||||||
* since replay detection used to detect DoS attacks may occasionally reject valid concurrent requests.
|
* since replay detection used to detect DoS attacks may occasionally reject valid concurrent requests.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testRequestIsAReplay(): Unit = {
|
||||||
def testRequestIsAReplay(quorum: String): Unit = {
|
|
||||||
val successfulAuthsPerThread = 10
|
val successfulAuthsPerThread = 10
|
||||||
val futures = (0 until numThreads).map(_ => executor.submit(new Runnable {
|
val futures = (0 until numThreads).map(_ => executor.submit(new Runnable {
|
||||||
override def run(): Unit = verifyRetriableFailuresDuringAuthentication(successfulAuthsPerThread)
|
override def run(): Unit = verifyRetriableFailuresDuringAuthentication(successfulAuthsPerThread)
|
||||||
|
@ -110,9 +109,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
|
||||||
* are able to connect after the second re-login. Verifies that logout is performed only once
|
* are able to connect after the second re-login. Verifies that logout is performed only once
|
||||||
* since duplicate logouts without successful login results in NPE from Java 9 onwards.
|
* since duplicate logouts without successful login results in NPE from Java 9 onwards.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testLoginFailure(): Unit = {
|
||||||
def testLoginFailure(quorum: String): Unit = {
|
|
||||||
val selector = createSelectorWithRelogin()
|
val selector = createSelectorWithRelogin()
|
||||||
try {
|
try {
|
||||||
val login = TestableKerberosLogin.instance
|
val login = TestableKerberosLogin.instance
|
||||||
|
@ -134,9 +132,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
|
||||||
* is performed when credentials are unavailable between logout and login, we handle it as a
|
* is performed when credentials are unavailable between logout and login, we handle it as a
|
||||||
* transient error and not an authentication failure so that clients may retry.
|
* transient error and not an authentication failure so that clients may retry.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testReLogin(): Unit = {
|
||||||
def testReLogin(quorum: String): Unit = {
|
|
||||||
val selector = createSelectorWithRelogin()
|
val selector = createSelectorWithRelogin()
|
||||||
try {
|
try {
|
||||||
val login = TestableKerberosLogin.instance
|
val login = TestableKerberosLogin.instance
|
||||||
|
@ -166,9 +163,8 @@ class GssapiAuthenticationTest extends IntegrationTestHarness with SaslSetup {
|
||||||
* Tests that Kerberos error `Server not found in Kerberos database (7)` is handled
|
* Tests that Kerberos error `Server not found in Kerberos database (7)` is handled
|
||||||
* as a fatal authentication failure.
|
* as a fatal authentication failure.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testServerNotFoundInKerberosDatabase(): Unit = {
|
||||||
def testServerNotFoundInKerberosDatabase(quorum: String): Unit = {
|
|
||||||
val jaasConfig = clientConfig.getProperty(SaslConfigs.SASL_JAAS_CONFIG)
|
val jaasConfig = clientConfig.getProperty(SaslConfigs.SASL_JAAS_CONFIG)
|
||||||
val invalidServiceConfig = jaasConfig.replace("serviceName=\"kafka\"", "serviceName=\"invalid-service\"")
|
val invalidServiceConfig = jaasConfig.replace("serviceName=\"kafka\"", "serviceName=\"invalid-service\"")
|
||||||
clientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, invalidServiceConfig)
|
clientConfig.put(SaslConfigs.SASL_JAAS_CONFIG, invalidServiceConfig)
|
||||||
|
|
|
@ -159,10 +159,6 @@ abstract class QuorumTestHarness extends Logging {
|
||||||
private var testInfo: TestInfo = _
|
private var testInfo: TestInfo = _
|
||||||
protected var implementation: QuorumImplementation = _
|
protected var implementation: QuorumImplementation = _
|
||||||
|
|
||||||
def isShareGroupTest(): Boolean = {
|
|
||||||
TestInfoUtils.isShareGroupTest(testInfo)
|
|
||||||
}
|
|
||||||
|
|
||||||
def maybeGroupProtocolSpecified(): Option[GroupProtocol] = {
|
def maybeGroupProtocolSpecified(): Option[GroupProtocol] = {
|
||||||
TestInfoUtils.maybeGroupProtocolSpecified(testInfo)
|
TestInfoUtils.maybeGroupProtocolSpecified(testInfo)
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,10 +34,6 @@ object TestInfoUtils {
|
||||||
|
|
||||||
final val TestWithParameterizedGroupProtocolNames = "{displayName}.groupProtocol={0}"
|
final val TestWithParameterizedGroupProtocolNames = "{displayName}.groupProtocol={0}"
|
||||||
|
|
||||||
def isShareGroupTest(testInfo: TestInfo): Boolean = {
|
|
||||||
testInfo.getDisplayName.contains("kip932")
|
|
||||||
}
|
|
||||||
|
|
||||||
def maybeGroupProtocolSpecified(testInfo: TestInfo): Option[GroupProtocol] = {
|
def maybeGroupProtocolSpecified(testInfo: TestInfo): Option[GroupProtocol] = {
|
||||||
if (testInfo.getDisplayName.contains("groupProtocol=classic"))
|
if (testInfo.getDisplayName.contains("groupProtocol=classic"))
|
||||||
Some(GroupProtocol.CLASSIC)
|
Some(GroupProtocol.CLASSIC)
|
||||||
|
|
|
@ -25,9 +25,7 @@ import org.apache.kafka.clients.admin.{Admin, NewPartitions, NewTopic}
|
||||||
import org.apache.kafka.common.errors.InvalidReplicaAssignmentException
|
import org.apache.kafka.common.errors.InvalidReplicaAssignmentException
|
||||||
import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse}
|
import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
import java.util.Arrays.asList
|
import java.util.Arrays.asList
|
||||||
|
@ -65,9 +63,8 @@ class AddPartitionsTest extends BaseRequestTest {
|
||||||
admin = createAdminClient()
|
admin = createAdminClient()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testWrongReplicaCount(): Unit = {
|
||||||
def testWrongReplicaCount(quorum: String): Unit = {
|
|
||||||
assertEquals(classOf[InvalidReplicaAssignmentException], assertThrows(classOf[ExecutionException], () => {
|
assertEquals(classOf[InvalidReplicaAssignmentException], assertThrows(classOf[ExecutionException], () => {
|
||||||
admin.createPartitions(Collections.singletonMap(topic1,
|
admin.createPartitions(Collections.singletonMap(topic1,
|
||||||
NewPartitions.increaseTo(2, singletonList(asList(0, 1, 2))))).all().get()
|
NewPartitions.increaseTo(2, singletonList(asList(0, 1, 2))))).all().get()
|
||||||
|
@ -78,9 +75,8 @@ class AddPartitionsTest extends BaseRequestTest {
|
||||||
* Test that when we supply a manual partition assignment to createTopics, it must be 0-based
|
* Test that when we supply a manual partition assignment to createTopics, it must be 0-based
|
||||||
* and consecutive.
|
* and consecutive.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testMissingPartitionsInCreateTopics(): Unit = {
|
||||||
def testMissingPartitionsInCreateTopics(quorum: String): Unit = {
|
|
||||||
val topic6Placements = new util.HashMap[Integer, util.List[Integer]]
|
val topic6Placements = new util.HashMap[Integer, util.List[Integer]]
|
||||||
topic6Placements.put(1, asList(0, 1))
|
topic6Placements.put(1, asList(0, 1))
|
||||||
topic6Placements.put(2, asList(1, 0))
|
topic6Placements.put(2, asList(1, 0))
|
||||||
|
@ -104,9 +100,8 @@ class AddPartitionsTest extends BaseRequestTest {
|
||||||
* Test that when we supply a manual partition assignment to createPartitions, it must contain
|
* Test that when we supply a manual partition assignment to createPartitions, it must contain
|
||||||
* enough partitions.
|
* enough partitions.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testMissingPartitionsInCreatePartitions(): Unit = {
|
||||||
def testMissingPartitionsInCreatePartitions(quorum: String): Unit = {
|
|
||||||
val cause = assertThrows(classOf[ExecutionException], () =>
|
val cause = assertThrows(classOf[ExecutionException], () =>
|
||||||
admin.createPartitions(Collections.singletonMap(topic1,
|
admin.createPartitions(Collections.singletonMap(topic1,
|
||||||
NewPartitions.increaseTo(3, singletonList(asList(0, 1, 2))))).all().get()).getCause
|
NewPartitions.increaseTo(3, singletonList(asList(0, 1, 2))))).all().get()).getCause
|
||||||
|
@ -115,9 +110,8 @@ class AddPartitionsTest extends BaseRequestTest {
|
||||||
"were specified."), "Unexpected error message: " + cause.getMessage)
|
"were specified."), "Unexpected error message: " + cause.getMessage)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIncrementPartitions(): Unit = {
|
||||||
def testIncrementPartitions(quorum: String): Unit = {
|
|
||||||
admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(3))).all().get()
|
admin.createPartitions(Collections.singletonMap(topic1, NewPartitions.increaseTo(3))).all().get()
|
||||||
|
|
||||||
// wait until leader is elected
|
// wait until leader is elected
|
||||||
|
@ -144,9 +138,8 @@ class AddPartitionsTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testManualAssignmentOfReplicas(): Unit = {
|
||||||
def testManualAssignmentOfReplicas(quorum: String): Unit = {
|
|
||||||
// Add 2 partitions
|
// Add 2 partitions
|
||||||
admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3,
|
admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3,
|
||||||
asList(asList(0, 1), asList(2, 3))))).all().get()
|
asList(asList(0, 1), asList(2, 3))))).all().get()
|
||||||
|
@ -173,9 +166,8 @@ class AddPartitionsTest extends BaseRequestTest {
|
||||||
assertEquals(Set(0, 1), replicas.asScala.toSet)
|
assertEquals(Set(0, 1), replicas.asScala.toSet)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testReplicaPlacementAllServers(): Unit = {
|
||||||
def testReplicaPlacementAllServers(quorum: String): Unit = {
|
|
||||||
admin.createPartitions(Collections.singletonMap(topic3, NewPartitions.increaseTo(7))).all().get()
|
admin.createPartitions(Collections.singletonMap(topic3, NewPartitions.increaseTo(7))).all().get()
|
||||||
|
|
||||||
// read metadata from a broker and verify the new topic partitions exist
|
// read metadata from a broker and verify the new topic partitions exist
|
||||||
|
@ -201,9 +193,8 @@ class AddPartitionsTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testReplicaPlacementPartialServers(): Unit = {
|
||||||
def testReplicaPlacementPartialServers(quorum: String): Unit = {
|
|
||||||
admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3))).all().get()
|
admin.createPartitions(Collections.singletonMap(topic2, NewPartitions.increaseTo(3))).all().get()
|
||||||
|
|
||||||
// read metadata from a broker and verify the new topic partitions exist
|
// read metadata from a broker and verify the new topic partitions exist
|
||||||
|
|
|
@ -1732,9 +1732,8 @@ class PartitionTest extends AbstractPartitionTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIsrNotExpandedIfReplicaIsFencedOrShutdown(): Unit = {
|
||||||
def testIsrNotExpandedIfReplicaIsFencedOrShutdown(quorum: String): Unit = {
|
|
||||||
val log = logManager.getOrCreateLog(topicPartition, topicId = topicId.toJava)
|
val log = logManager.getOrCreateLog(topicPartition, topicId = topicId.toJava)
|
||||||
seedLogData(log, numRecords = 10, leaderEpoch = 4)
|
seedLogData(log, numRecords = 10, leaderEpoch = 4)
|
||||||
|
|
||||||
|
|
|
@ -23,17 +23,15 @@ import scala.collection.Seq
|
||||||
import kafka.server.KafkaConfig
|
import kafka.server.KafkaConfig
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import org.apache.kafka.server.config.ServerLogConfigs
|
import org.apache.kafka.server.config.ServerLogConfigs
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
class MinIsrConfigTest extends KafkaServerTestHarness {
|
class MinIsrConfigTest extends KafkaServerTestHarness {
|
||||||
val overridingProps = new Properties()
|
val overridingProps = new Properties()
|
||||||
overridingProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "5")
|
overridingProps.put(ServerLogConfigs.MIN_IN_SYNC_REPLICAS_CONFIG, "5")
|
||||||
def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1).map(KafkaConfig.fromProps(_, overridingProps))
|
def generateConfigs: Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1).map(KafkaConfig.fromProps(_, overridingProps))
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDefaultKafkaConfig(): Unit = {
|
||||||
def testDefaultKafkaConfig(quorum: String): Unit = {
|
|
||||||
assert(brokers.head.logManager.initialDefaultConfig.minInSyncReplicas == 5)
|
assert(brokers.head.logManager.initialDefaultConfig.minInSyncReplicas == 5)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,9 +36,9 @@ import org.apache.kafka.common.utils.Time
|
||||||
import org.apache.kafka.server.config.ServerLogConfigs
|
import org.apache.kafka.server.config.ServerLogConfigs
|
||||||
import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector}
|
import org.apache.kafka.server.metrics.{KafkaMetricsGroup, KafkaYammerMetrics, LinuxIoMetricsCollector}
|
||||||
import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics
|
import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics
|
||||||
import org.junit.jupiter.api.Timeout
|
import org.junit.jupiter.api.{Test, Timeout}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
|
import org.junit.jupiter.params.provider.MethodSource
|
||||||
|
|
||||||
@Timeout(120)
|
@Timeout(120)
|
||||||
class MetricsTest extends KafkaServerTestHarness with Logging {
|
class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
|
@ -56,9 +56,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
|
|
||||||
val nMessages = 2
|
val nMessages = 2
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testMetricsReporterAfterDeletingTopic(): Unit = {
|
||||||
def testMetricsReporterAfterDeletingTopic(quorum: String): Unit = {
|
|
||||||
val topic = "test-topic-metric"
|
val topic = "test-topic-metric"
|
||||||
createTopic(topic)
|
createTopic(topic)
|
||||||
deleteTopic(topic)
|
deleteTopic(topic)
|
||||||
|
@ -66,9 +65,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic")
|
assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testBrokerTopicMetricsUnregisteredAfterDeletingTopic(): Unit = {
|
||||||
def testBrokerTopicMetricsUnregisteredAfterDeletingTopic(quorum: String): Unit = {
|
|
||||||
val topic = "test-broker-topic-metric"
|
val topic = "test-broker-topic-metric"
|
||||||
createTopic(topic, 2)
|
createTopic(topic, 2)
|
||||||
// Produce a few messages to create the metrics
|
// Produce a few messages to create the metrics
|
||||||
|
@ -81,33 +79,29 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic")
|
assertEquals(Set.empty, topicMetricGroups(topic), "Topic metrics exists after deleteTopic")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testClusterIdMetric(): Unit = {
|
||||||
def testClusterIdMetric(quorum: String): Unit = {
|
|
||||||
// Check if clusterId metric exists.
|
// Check if clusterId metric exists.
|
||||||
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
||||||
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=ClusterId"), 1)
|
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=ClusterId"), 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testBrokerStateMetric(): Unit = {
|
||||||
def testBrokerStateMetric(quorum: String): Unit = {
|
|
||||||
// Check if BrokerState metric exists.
|
// Check if BrokerState metric exists.
|
||||||
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
||||||
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=BrokerState"), 1)
|
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=BrokerState"), 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testYammerMetricsCountMetric(): Unit = {
|
||||||
def testYammerMetricsCountMetric(quorum: String): Unit = {
|
|
||||||
// Check if yammer-metrics-count metric exists.
|
// Check if yammer-metrics-count metric exists.
|
||||||
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
||||||
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=yammer-metrics-count"), 1)
|
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=yammer-metrics-count"), 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testLinuxIoMetrics(): Unit = {
|
||||||
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
|
// 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.
|
// able to collect those metrics on the platform where this test is running.
|
||||||
val usable = new LinuxIoMetricsCollector("/proc", Time.SYSTEM).usable()
|
val usable = new LinuxIoMetricsCollector("/proc", Time.SYSTEM).usable()
|
||||||
|
@ -117,9 +111,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=$name"), expectedCount))
|
assertEquals(metrics.keySet.asScala.count(_.getMBeanName == s"$requiredKafkaServerPrefix=$name"), expectedCount))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testJMXFilter(): Unit = {
|
||||||
def testJMXFilter(quorum: String): Unit = {
|
|
||||||
// Check if cluster id metrics is not exposed in JMX
|
// Check if cluster id metrics is not exposed in JMX
|
||||||
assertTrue(ManagementFactory.getPlatformMBeanServer
|
assertTrue(ManagementFactory.getPlatformMBeanServer
|
||||||
.isRegistered(new ObjectName("kafka.controller:type=KafkaController,name=ActiveControllerCount")))
|
.isRegistered(new ObjectName("kafka.controller:type=KafkaController,name=ActiveControllerCount")))
|
||||||
|
@ -127,9 +120,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
.isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId")))
|
.isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId")))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUpdateJMXFilter(): Unit = {
|
||||||
def testUpdateJMXFilter(quorum: String): Unit = {
|
|
||||||
// verify previously exposed metrics are removed and existing matching metrics are added
|
// verify previously exposed metrics are removed and existing matching metrics are added
|
||||||
brokers.foreach(broker => broker.kafkaYammerMetrics.reconfigure(
|
brokers.foreach(broker => broker.kafkaYammerMetrics.reconfigure(
|
||||||
Map(JmxReporter.EXCLUDE_CONFIG -> "kafka.controller:type=KafkaController,name=ActiveControllerCount").asJava
|
Map(JmxReporter.EXCLUDE_CONFIG -> "kafka.controller:type=KafkaController,name=ActiveControllerCount").asJava
|
||||||
|
@ -140,9 +132,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
.isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId")))
|
.isRegistered(new ObjectName(s"$requiredKafkaServerPrefix=ClusterId")))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testGeneralBrokerTopicMetricsAreGreedilyRegistered(): Unit = {
|
||||||
def testGeneralBrokerTopicMetricsAreGreedilyRegistered(quorum: String): Unit = {
|
|
||||||
val topic = "test-broker-topic-metric"
|
val topic = "test-broker-topic-metric"
|
||||||
createTopic(topic, 2)
|
createTopic(topic, 2)
|
||||||
|
|
||||||
|
@ -156,9 +147,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
assertTrue(topicMetricGroups(topic).nonEmpty, "Topic metrics aren't registered")
|
assertTrue(topicMetricGroups(topic).nonEmpty, "Topic metrics aren't registered")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testWindowsStyleTagNames(): Unit = {
|
||||||
def testWindowsStyleTagNames(quorum: String): Unit = {
|
|
||||||
val path = "C:\\windows-path\\kafka-logs"
|
val path = "C:\\windows-path\\kafka-logs"
|
||||||
val tags = Map("dir" -> path)
|
val tags = Map("dir" -> path)
|
||||||
val expectedMBeanName = Set(tags.keySet.head, ObjectName.quote(path)).mkString("=")
|
val expectedMBeanName = Set(tags.keySet.head, ObjectName.quote(path)).mkString("=")
|
||||||
|
@ -213,9 +203,8 @@ class MetricsTest extends KafkaServerTestHarness with Logging {
|
||||||
assertTrue(TestUtils.meterCount(bytesOut) > initialBytesOut)
|
assertTrue(TestUtils.meterCount(bytesOut) > initialBytesOut)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testKRaftControllerMetrics(): Unit = {
|
||||||
def testKRaftControllerMetrics(quorum: String): Unit = {
|
|
||||||
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
val metrics = KafkaYammerMetrics.defaultRegistry.allMetrics
|
||||||
Set(
|
Set(
|
||||||
"kafka.controller:type=KafkaController,name=ActiveControllerCount",
|
"kafka.controller:type=KafkaController,name=ActiveControllerCount",
|
||||||
|
|
|
@ -41,8 +41,6 @@ import org.apache.kafka.server.authorizer._
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
||||||
import org.junit.jupiter.api.Test
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.net.InetAddress
|
import java.net.InetAddress
|
||||||
import java.util
|
import java.util
|
||||||
|
@ -52,7 +50,6 @@ import scala.jdk.CollectionConverters._
|
||||||
class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
|
|
||||||
private final val PLAINTEXT = new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "127.0.0.1", 9020)
|
private final val PLAINTEXT = new Endpoint("PLAINTEXT", SecurityProtocol.PLAINTEXT, "127.0.0.1", 9020)
|
||||||
private final val KRAFT = "kraft"
|
|
||||||
|
|
||||||
private val allowReadAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, READ, ALLOW)
|
private val allowReadAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, READ, ALLOW)
|
||||||
private val allowWriteAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, WRITE, ALLOW)
|
private val allowWriteAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, WRITE, ALLOW)
|
||||||
|
@ -104,32 +101,28 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
super.tearDown()
|
super.tearDown()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAuthorizeThrowsOnNonLiteralResource(): Unit = {
|
||||||
def testAuthorizeThrowsOnNonLiteralResource(quorum: String): Unit = {
|
|
||||||
assertThrows(classOf[IllegalArgumentException], () => authorize(authorizer1, requestContext, READ,
|
assertThrows(classOf[IllegalArgumentException], () => authorize(authorizer1, requestContext, READ,
|
||||||
new ResourcePattern(TOPIC, "something", PREFIXED)))
|
new ResourcePattern(TOPIC, "something", PREFIXED)))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAuthorizeWithEmptyResourceName(): Unit = {
|
||||||
def testAuthorizeWithEmptyResourceName(quorum: String): Unit = {
|
|
||||||
assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(GROUP, "", LITERAL)))
|
assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(GROUP, "", LITERAL)))
|
||||||
addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(GROUP, WILDCARD_RESOURCE, LITERAL))
|
addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(GROUP, WILDCARD_RESOURCE, LITERAL))
|
||||||
assertTrue(authorize(authorizer1, requestContext, READ, new ResourcePattern(GROUP, "", LITERAL)))
|
assertTrue(authorize(authorizer1, requestContext, READ, new ResourcePattern(GROUP, "", LITERAL)))
|
||||||
}
|
}
|
||||||
|
|
||||||
// Authorizing the empty resource is not supported because empty resource name is invalid.
|
// Authorizing the empty resource is not supported because empty resource name is invalid.
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testEmptyAclThrowsException(): Unit = {
|
||||||
def testEmptyAclThrowsException(quorum: String): Unit = {
|
|
||||||
assertThrows(classOf[ApiException],
|
assertThrows(classOf[ApiException],
|
||||||
() => addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(GROUP, "", LITERAL)))
|
() => addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(GROUP, "", LITERAL)))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testTopicAcl(): Unit = {
|
||||||
def testTopicAcl(quorum: String): Unit = {
|
|
||||||
val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||||
val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "rob")
|
val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "rob")
|
||||||
val user3 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "batman")
|
val user3 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "batman")
|
||||||
|
@ -183,9 +176,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
/**
|
/**
|
||||||
* CustomPrincipals should be compared with their principal type and name
|
* CustomPrincipals should be compared with their principal type and name
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAllowAccessWithCustomPrincipal(): Unit = {
|
||||||
def testAllowAccessWithCustomPrincipal(quorum: String): Unit = {
|
|
||||||
val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||||
val customUserPrincipal = new CustomPrincipal(KafkaPrincipal.USER_TYPE, username)
|
val customUserPrincipal = new CustomPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||||
val host1 = InetAddress.getByName("192.168.1.1")
|
val host1 = InetAddress.getByName("192.168.1.1")
|
||||||
|
@ -204,9 +196,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertFalse(authorize(authorizer1, host1Context, READ, resource), "User1 should not have READ access from host1 due to denyAcl")
|
assertFalse(authorize(authorizer1, host1Context, READ, resource), "User1 should not have READ access from host1 due to denyAcl")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testDenyTakesPrecedence(): Unit = {
|
||||||
def testDenyTakesPrecedence(quorum: String): Unit = {
|
|
||||||
val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
val user = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||||
val host = InetAddress.getByName("192.168.2.1")
|
val host = InetAddress.getByName("192.168.2.1")
|
||||||
val session = newRequestContext(user, host)
|
val session = newRequestContext(user, host)
|
||||||
|
@ -220,9 +211,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertFalse(authorize(authorizer1, session, READ, resource), "deny should take precedence over allow.")
|
assertFalse(authorize(authorizer1, session, READ, resource), "deny should take precedence over allow.")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAllowAllAccess(): Unit = {
|
||||||
def testAllowAllAccess(quorum: String): Unit = {
|
|
||||||
val allowAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, ALLOW)
|
val allowAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, ALLOW)
|
||||||
|
|
||||||
changeAclAndVerify(Set.empty, Set(allowAllAcl), Set.empty)
|
changeAclAndVerify(Set.empty, Set(allowAllAcl), Set.empty)
|
||||||
|
@ -231,9 +221,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertTrue(authorize(authorizer1, context, READ, resource), "allow all acl should allow access to all.")
|
assertTrue(authorize(authorizer1, context, READ, resource), "allow all acl should allow access to all.")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testSuperUserHasAccess(): Unit = {
|
||||||
def testSuperUserHasAccess(quorum: String): Unit = {
|
|
||||||
val denyAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, DENY)
|
val denyAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, DENY)
|
||||||
|
|
||||||
changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty)
|
changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty)
|
||||||
|
@ -248,9 +237,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
/**
|
/**
|
||||||
* CustomPrincipals should be compared with their principal type and name
|
* CustomPrincipals should be compared with their principal type and name
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testSuperUserWithCustomPrincipalHasAccess(): Unit = {
|
||||||
def testSuperUserWithCustomPrincipalHasAccess(quorum: String): Unit = {
|
|
||||||
val denyAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, DENY)
|
val denyAllAcl = new AccessControlEntry(WILDCARD_PRINCIPAL_STRING, WILDCARD_HOST, AclOperation.ALL, DENY)
|
||||||
changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty)
|
changeAclAndVerify(Set.empty, Set(denyAllAcl), Set.empty)
|
||||||
|
|
||||||
|
@ -259,9 +247,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertTrue(authorize(authorizer1, session, READ, resource), "superuser with custom principal always has access, no matter what acls.")
|
assertTrue(authorize(authorizer1, session, READ, resource), "superuser with custom principal always has access, no matter what acls.")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testWildCardAcls(): Unit = {
|
||||||
def testWildCardAcls(quorum: String): Unit = {
|
|
||||||
assertFalse(authorize(authorizer1, requestContext, READ, resource), "when acls = [], authorizer should fail close.")
|
assertFalse(authorize(authorizer1, requestContext, READ, resource), "when acls = [], authorizer should fail close.")
|
||||||
|
|
||||||
val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||||
|
@ -284,15 +271,13 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertFalse(authorize(authorizer1, host1Context, WRITE, resource), "User1 should not have WRITE access from host1")
|
assertFalse(authorize(authorizer1, host1Context, WRITE, resource), "User1 should not have WRITE access from host1")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testNoAclFound(): Unit = {
|
||||||
def testNoAclFound(quorum: String): Unit = {
|
|
||||||
assertFalse(authorize(authorizer1, requestContext, READ, resource), "when acls = [], authorizer should deny op.")
|
assertFalse(authorize(authorizer1, requestContext, READ, resource), "when acls = [], authorizer should deny op.")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testNoAclFoundOverride(): Unit = {
|
||||||
def testNoAclFoundOverride(quorum: String): Unit = {
|
|
||||||
val props = properties
|
val props = properties
|
||||||
props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true")
|
props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true")
|
||||||
|
|
||||||
|
@ -307,9 +292,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAclConfigWithWhitespace(): Unit = {
|
||||||
def testAclConfigWithWhitespace(quorum: String): Unit = {
|
|
||||||
val props = properties
|
val props = properties
|
||||||
props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, " true")
|
props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, " true")
|
||||||
// replace all property values with leading & trailing whitespaces
|
// replace all property values with leading & trailing whitespaces
|
||||||
|
@ -325,9 +309,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAclManagementAPIs(): Unit = {
|
||||||
def testAclManagementAPIs(quorum: String): Unit = {
|
|
||||||
val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username)
|
||||||
val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob")
|
val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob")
|
||||||
val host1 = "host1"
|
val host1 = "host1"
|
||||||
|
@ -393,9 +376,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
/**
|
/**
|
||||||
* Test ACL inheritance, as described in #{org.apache.kafka.common.acl.AclOperation}
|
* Test ACL inheritance, as described in #{org.apache.kafka.common.acl.AclOperation}
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAclInheritance(): Unit = {
|
||||||
def testAclInheritance(quorum: String): Unit = {
|
|
||||||
testImplicationsOfAllow(AclOperation.ALL, Set(READ, WRITE, CREATE, DELETE, ALTER, DESCRIBE,
|
testImplicationsOfAllow(AclOperation.ALL, Set(READ, WRITE, CREATE, DELETE, ALTER, DESCRIBE,
|
||||||
CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, CREATE_TOKENS, DESCRIBE_TOKENS, TWO_PHASE_COMMIT))
|
CLUSTER_ACTION, DESCRIBE_CONFIGS, ALTER_CONFIGS, IDEMPOTENT_WRITE, CREATE_TOKENS, DESCRIBE_TOKENS, TWO_PHASE_COMMIT))
|
||||||
testImplicationsOfDeny(AclOperation.ALL, Set(READ, WRITE, CREATE, DELETE, ALTER, DESCRIBE,
|
testImplicationsOfDeny(AclOperation.ALL, Set(READ, WRITE, CREATE, DELETE, ALTER, DESCRIBE,
|
||||||
|
@ -442,17 +424,15 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
removeAcls(authorizer1, acls, clusterResource)
|
removeAcls(authorizer1, acls, clusterResource)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAccessAllowedIfAllowAclExistsOnWildcardResource(): Unit = {
|
||||||
def testAccessAllowedIfAllowAclExistsOnWildcardResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl), wildCardResource)
|
addAcls(authorizer1, Set(allowReadAcl), wildCardResource)
|
||||||
|
|
||||||
assertTrue(authorize(authorizer1, requestContext, READ, resource))
|
assertTrue(authorize(authorizer1, requestContext, READ, resource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testDeleteAclOnWildcardResource(): Unit = {
|
||||||
def testDeleteAclOnWildcardResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), wildCardResource)
|
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), wildCardResource)
|
||||||
|
|
||||||
removeAcls(authorizer1, Set(allowReadAcl), wildCardResource)
|
removeAcls(authorizer1, Set(allowReadAcl), wildCardResource)
|
||||||
|
@ -460,9 +440,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set(allowWriteAcl), getAcls(authorizer1, wildCardResource))
|
assertEquals(Set(allowWriteAcl), getAcls(authorizer1, wildCardResource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testDeleteAllAclOnWildcardResource(): Unit = {
|
||||||
def testDeleteAllAclOnWildcardResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl), wildCardResource)
|
addAcls(authorizer1, Set(allowReadAcl), wildCardResource)
|
||||||
|
|
||||||
removeAcls(authorizer1, Set.empty, wildCardResource)
|
removeAcls(authorizer1, Set.empty, wildCardResource)
|
||||||
|
@ -470,17 +449,15 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set.empty, getAcls(authorizer1))
|
assertEquals(Set.empty, getAcls(authorizer1))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAccessAllowedIfAllowAclExistsOnPrefixedResource(): Unit = {
|
||||||
def testAccessAllowedIfAllowAclExistsOnPrefixedResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl), prefixedResource)
|
addAcls(authorizer1, Set(allowReadAcl), prefixedResource)
|
||||||
|
|
||||||
assertTrue(authorize(authorizer1, requestContext, READ, resource))
|
assertTrue(authorize(authorizer1, requestContext, READ, resource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testDeleteAclOnPrefixedResource(): Unit = {
|
||||||
def testDeleteAclOnPrefixedResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource)
|
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource)
|
||||||
|
|
||||||
removeAcls(authorizer1, Set(allowReadAcl), prefixedResource)
|
removeAcls(authorizer1, Set(allowReadAcl), prefixedResource)
|
||||||
|
@ -488,9 +465,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set(allowWriteAcl), getAcls(authorizer1, prefixedResource))
|
assertEquals(Set(allowWriteAcl), getAcls(authorizer1, prefixedResource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testDeleteAllAclOnPrefixedResource(): Unit = {
|
||||||
def testDeleteAllAclOnPrefixedResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource)
|
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource)
|
||||||
|
|
||||||
removeAcls(authorizer1, Set.empty, prefixedResource)
|
removeAcls(authorizer1, Set.empty, prefixedResource)
|
||||||
|
@ -498,9 +474,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set.empty, getAcls(authorizer1))
|
assertEquals(Set.empty, getAcls(authorizer1))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAddAclsOnLiteralResource(): Unit = {
|
||||||
def testAddAclsOnLiteralResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), resource)
|
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), resource)
|
||||||
addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), resource)
|
addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), resource)
|
||||||
|
|
||||||
|
@ -509,9 +484,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set.empty, getAcls(authorizer1, prefixedResource))
|
assertEquals(Set.empty, getAcls(authorizer1, prefixedResource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAddAclsOnWildcardResource(): Unit = {
|
||||||
def testAddAclsOnWildcardResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), wildCardResource)
|
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), wildCardResource)
|
||||||
addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), wildCardResource)
|
addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), wildCardResource)
|
||||||
|
|
||||||
|
@ -520,9 +494,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set.empty, getAcls(authorizer1, prefixedResource))
|
assertEquals(Set.empty, getAcls(authorizer1, prefixedResource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAddAclsOnPrefixedResource(): Unit = {
|
||||||
def testAddAclsOnPrefixedResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource)
|
addAcls(authorizer1, Set(allowReadAcl, allowWriteAcl), prefixedResource)
|
||||||
addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), prefixedResource)
|
addAcls(authorizer1, Set(allowWriteAcl, denyReadAcl), prefixedResource)
|
||||||
|
|
||||||
|
@ -531,9 +504,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set.empty, getAcls(authorizer1, resource))
|
assertEquals(Set.empty, getAcls(authorizer1, resource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAuthorizeWithPrefixedResource(): Unit = {
|
||||||
def testAuthorizeWithPrefixedResource(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "a_other", LITERAL))
|
addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "a_other", LITERAL))
|
||||||
addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "a_other", PREFIXED))
|
addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "a_other", PREFIXED))
|
||||||
addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), PREFIXED))
|
addAcls(authorizer1, Set(denyReadAcl), new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), PREFIXED))
|
||||||
|
@ -552,9 +524,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertTrue(authorize(authorizer1, requestContext, READ, resource))
|
assertTrue(authorize(authorizer1, requestContext, READ, resource))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testSingleCharacterResourceAcls(): Unit = {
|
||||||
def testSingleCharacterResourceAcls(quorum: String): Unit = {
|
|
||||||
addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(TOPIC, "f", LITERAL))
|
addAcls(authorizer1, Set(allowReadAcl), new ResourcePattern(TOPIC, "f", LITERAL))
|
||||||
assertTrue(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "f", LITERAL)))
|
assertTrue(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "f", LITERAL)))
|
||||||
assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "foo", LITERAL)))
|
assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "foo", LITERAL)))
|
||||||
|
@ -565,9 +536,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "foo_", LITERAL)))
|
assertFalse(authorize(authorizer1, requestContext, READ, new ResourcePattern(TOPIC, "foo_", LITERAL)))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testGetAclsPrincipal(): Unit = {
|
||||||
def testGetAclsPrincipal(quorum: String): Unit = {
|
|
||||||
val aclOnSpecificPrincipal = new AccessControlEntry(principal.toString, WILDCARD_HOST, WRITE, ALLOW)
|
val aclOnSpecificPrincipal = new AccessControlEntry(principal.toString, WILDCARD_HOST, WRITE, ALLOW)
|
||||||
addAcls(authorizer1, Set(aclOnSpecificPrincipal), resource)
|
addAcls(authorizer1, Set(aclOnSpecificPrincipal), resource)
|
||||||
|
|
||||||
|
@ -586,9 +556,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(0, getAcls(authorizer1, principal).size, "acl on wildcard should not be returned for specific request")
|
assertEquals(0, getAcls(authorizer1, principal).size, "acl on wildcard should not be returned for specific request")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAclsFilter(): Unit = {
|
||||||
def testAclsFilter(quorum: String): Unit = {
|
|
||||||
val resource1 = new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), LITERAL)
|
val resource1 = new ResourcePattern(TOPIC, "foo-" + UUID.randomUUID(), LITERAL)
|
||||||
val resource2 = new ResourcePattern(TOPIC, "bar-" + UUID.randomUUID(), LITERAL)
|
val resource2 = new ResourcePattern(TOPIC, "bar-" + UUID.randomUUID(), LITERAL)
|
||||||
val prefixedResource = new ResourcePattern(TOPIC, "bar-", PREFIXED)
|
val prefixedResource = new ResourcePattern(TOPIC, "bar-", PREFIXED)
|
||||||
|
@ -622,9 +591,8 @@ class AuthorizerTest extends QuorumTestHarness with BaseAuthorizerTest {
|
||||||
assertEquals(Set.empty, deleteResults(3).aclBindingDeleteResults.asScala.map(_.aclBinding).toSet)
|
assertEquals(Set.empty, deleteResults(3).aclBindingDeleteResults.asScala.map(_.aclBinding).toSet)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array(KRAFT))
|
def testAuthorizeByResourceTypeNoAclFoundOverride(): Unit = {
|
||||||
def testAuthorizeByResourceTypeNoAclFoundOverride(quorum: String): Unit = {
|
|
||||||
val props = properties
|
val props = properties
|
||||||
props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true")
|
props.put(StandardAuthorizer.ALLOW_EVERYONE_IF_NO_ACL_IS_FOUND_CONFIG, "true")
|
||||||
|
|
||||||
|
|
|
@ -32,9 +32,9 @@ import org.apache.kafka.common.requests.FindCoordinatorRequest.CoordinatorType
|
||||||
import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, FindCoordinatorRequest, FindCoordinatorResponse, InitProducerIdRequest, InitProducerIdResponse}
|
import org.apache.kafka.common.requests.{AddPartitionsToTxnRequest, AddPartitionsToTxnResponse, FindCoordinatorRequest, FindCoordinatorResponse, InitProducerIdRequest, InitProducerIdResponse}
|
||||||
import org.apache.kafka.server.config.ServerLogConfigs
|
import org.apache.kafka.server.config.ServerLogConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.{Arguments, MethodSource, ValueSource}
|
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
|
||||||
|
|
||||||
import scala.collection.mutable
|
import scala.collection.mutable
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -55,7 +55,7 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
|
||||||
|
|
||||||
@ParameterizedTest
|
@ParameterizedTest
|
||||||
@MethodSource(value = Array("parameters"))
|
@MethodSource(value = Array("parameters"))
|
||||||
def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(quorum: String, version: Short): Unit = {
|
def shouldReceiveOperationNotAttemptedWhenOtherPartitionHasError(version: Short): Unit = {
|
||||||
// The basic idea is that we have one unknown topic and one created topic. We should get the 'UNKNOWN_TOPIC_OR_PARTITION'
|
// The basic idea is that we have one unknown topic and one created topic. We should get the 'UNKNOWN_TOPIC_OR_PARTITION'
|
||||||
// error for the unknown topic and the 'OPERATION_NOT_ATTEMPTED' error for the known and authorized topic.
|
// error for the unknown topic and the 'OPERATION_NOT_ATTEMPTED' error for the known and authorized topic.
|
||||||
val nonExistentTopic = new TopicPartition("unknownTopic", 0)
|
val nonExistentTopic = new TopicPartition("unknownTopic", 0)
|
||||||
|
@ -110,9 +110,8 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
|
||||||
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
|
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, errors.get(nonExistentTopic))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testOneSuccessOneErrorInBatchedRequest(): Unit = {
|
||||||
def testOneSuccessOneErrorInBatchedRequest(quorum: String): Unit = {
|
|
||||||
val tp0 = new TopicPartition(topic1, 0)
|
val tp0 = new TopicPartition(topic1, 0)
|
||||||
val transactionalId1 = "foobar"
|
val transactionalId1 = "foobar"
|
||||||
val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
|
val transactionalId2 = "barfoo" // "barfoo" maps to the same transaction coordinator
|
||||||
|
@ -149,9 +148,8 @@ class AddPartitionsToTxnRequestServerTest extends BaseRequestTest {
|
||||||
assertEquals(expectedErrors, errors)
|
assertEquals(expectedErrors, errors)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testVerifyOnly(): Unit = {
|
||||||
def testVerifyOnly(quorum: String): Unit = {
|
|
||||||
val tp0 = new TopicPartition(topic1, 0)
|
val tp0 = new TopicPartition(topic1, 0)
|
||||||
|
|
||||||
val transactionalId = "foobar"
|
val transactionalId = "foobar"
|
||||||
|
@ -209,7 +207,7 @@ object AddPartitionsToTxnRequestServerTest {
|
||||||
def parameters: JStream[Arguments] = {
|
def parameters: JStream[Arguments] = {
|
||||||
val arguments = mutable.ListBuffer[Arguments]()
|
val arguments = mutable.ListBuffer[Arguments]()
|
||||||
ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions().forEach { version =>
|
ApiKeys.ADD_PARTITIONS_TO_TXN.allVersions().forEach { version =>
|
||||||
arguments += Arguments.of("kraft", version)
|
arguments += Arguments.of(version)
|
||||||
}
|
}
|
||||||
arguments.asJava.stream()
|
arguments.asJava.stream()
|
||||||
}
|
}
|
||||||
|
|
|
@ -27,8 +27,7 @@ import org.apache.kafka.common.requests.{AlterReplicaLogDirsRequest, AlterReplic
|
||||||
import org.apache.kafka.server.config.ServerLogConfigs
|
import org.apache.kafka.server.config.ServerLogConfigs
|
||||||
import org.apache.kafka.storage.internals.log.LogFileUtils
|
import org.apache.kafka.storage.internals.log.LogFileUtils
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -52,9 +51,8 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
|
||||||
.find(p => p.partitionIndex == tp.partition).get.errorCode)
|
.find(p => p.partitionIndex == tp.partition).get.errorCode)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterReplicaLogDirsRequest(): Unit = {
|
||||||
def testAlterReplicaLogDirsRequest(quorum: String): Unit = {
|
|
||||||
val partitionNum = 5
|
val partitionNum = 5
|
||||||
|
|
||||||
// Alter replica dir before topic creation
|
// Alter replica dir before topic creation
|
||||||
|
@ -88,9 +86,8 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterReplicaLogDirsRequestErrorCode(): Unit = {
|
||||||
def testAlterReplicaLogDirsRequestErrorCode(quorum: String): Unit = {
|
|
||||||
val offlineDir = new File(brokers.head.config.logDirs.tail.head).getAbsolutePath
|
val offlineDir = new File(brokers.head.config.logDirs.tail.head).getAbsolutePath
|
||||||
val validDir1 = new File(brokers.head.config.logDirs(1)).getAbsolutePath
|
val validDir1 = new File(brokers.head.config.logDirs(1)).getAbsolutePath
|
||||||
val validDir2 = new File(brokers.head.config.logDirs(2)).getAbsolutePath
|
val validDir2 = new File(brokers.head.config.logDirs(2)).getAbsolutePath
|
||||||
|
@ -127,9 +124,8 @@ class AlterReplicaLogDirsRequestTest extends BaseRequestTest {
|
||||||
assertEquals(Errors.KAFKA_STORAGE_ERROR, findErrorForPartition(alterReplicaDirResponse3, new TopicPartition(topic, 2)))
|
assertEquals(Errors.KAFKA_STORAGE_ERROR, findErrorForPartition(alterReplicaDirResponse3, new TopicPartition(topic, 2)))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterReplicaLogDirsRequestWithRetention(): Unit = {
|
||||||
def testAlterReplicaLogDirsRequestWithRetention(quorum: String): Unit = {
|
|
||||||
val partitionNum = 1
|
val partitionNum = 1
|
||||||
|
|
||||||
// Alter replica dir before topic creation
|
// Alter replica dir before topic creation
|
||||||
|
|
|
@ -25,9 +25,7 @@ import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse}
|
import org.apache.kafka.common.requests.{AlterUserScramCredentialsRequest, AlterUserScramCredentialsResponse}
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.TestInfo
|
import org.junit.jupiter.api.{Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
@ -54,9 +52,8 @@ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest
|
||||||
private val user1 = "user1"
|
private val user1 = "user1"
|
||||||
private val user2 = "user2"
|
private val user2 = "user2"
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterNothingNotAuthorized(): Unit = {
|
||||||
def testAlterNothingNotAuthorized(quorum: String): Unit = {
|
|
||||||
val request = new AlterUserScramCredentialsRequest.Builder(
|
val request = new AlterUserScramCredentialsRequest.Builder(
|
||||||
new AlterUserScramCredentialsRequestData()
|
new AlterUserScramCredentialsRequestData()
|
||||||
.setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
|
.setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
|
||||||
|
@ -67,9 +64,8 @@ class AlterUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTest
|
||||||
assertEquals(0, results.size)
|
assertEquals(0, results.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterSomethingNotAuthorized(): Unit = {
|
||||||
def testAlterSomethingNotAuthorized(quorum: String): Unit = {
|
|
||||||
val request = new AlterUserScramCredentialsRequest.Builder(
|
val request = new AlterUserScramCredentialsRequest.Builder(
|
||||||
new AlterUserScramCredentialsRequestData()
|
new AlterUserScramCredentialsRequestData()
|
||||||
.setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
|
.setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
|
||||||
|
|
|
@ -34,7 +34,7 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild
|
||||||
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
|
import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, AuthorizationResult}
|
||||||
import org.apache.kafka.server.common.MetadataVersion
|
import org.apache.kafka.server.common.MetadataVersion
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
import org.junit.jupiter.params.provider.ValueSource
|
||||||
|
@ -79,9 +79,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
private val user3 = "user3@user3.com"
|
private val user3 = "user3@user3.com"
|
||||||
private val unknownUser = "unknownUser"
|
private val unknownUser = "unknownUser"
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterNothing(): Unit = {
|
||||||
def testAlterNothing(quorum: String): Unit = {
|
|
||||||
val request = new AlterUserScramCredentialsRequest.Builder(
|
val request = new AlterUserScramCredentialsRequest.Builder(
|
||||||
new AlterUserScramCredentialsRequestData()
|
new AlterUserScramCredentialsRequestData()
|
||||||
.setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
|
.setDeletions(new util.ArrayList[AlterUserScramCredentialsRequestData.ScramCredentialDeletion])
|
||||||
|
@ -92,9 +91,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
assertEquals(0, results.size)
|
assertEquals(0, results.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterSameThingTwice(): Unit = {
|
||||||
def testAlterSameThingTwice(quorum: String): Unit = {
|
|
||||||
val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
val deletion1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||||
val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
val deletion2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||||
val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
val upsertion1 = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||||
|
@ -133,9 +131,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterEmptyUser(): Unit = {
|
||||||
def testAlterEmptyUser(quorum: String): Unit = {
|
|
||||||
val deletionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
val deletionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||||
val upsertionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
val upsertionEmpty = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName("").setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||||
.setIterations(4096).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
.setIterations(4096).setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||||
|
@ -162,9 +159,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterUnknownMechanism(): Unit = {
|
||||||
def testAlterUnknownMechanism(quorum: String): Unit = {
|
|
||||||
val deletionUnknown1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.UNKNOWN.`type`)
|
val deletionUnknown1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.UNKNOWN.`type`)
|
||||||
val deletionValid1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
val deletionValid1 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)
|
||||||
val deletionUnknown2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(10.toByte)
|
val deletionUnknown2 = new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user2).setMechanism(10.toByte)
|
||||||
|
@ -190,9 +186,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
results.asScala.foreach(result => assertEquals("Unknown SCRAM mechanism", result.errorMessage))
|
results.asScala.foreach(result => assertEquals("Unknown SCRAM mechanism", result.errorMessage))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterTooFewIterations(): Unit = {
|
||||||
def testAlterTooFewIterations(quorum: String): Unit = {
|
|
||||||
val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1)
|
val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1)
|
||||||
.setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(1)
|
.setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(1)
|
||||||
.setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
.setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||||
|
@ -207,9 +202,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
assertEquals("Too few iterations", results.get(0).errorMessage)
|
assertEquals("Too few iterations", results.get(0).errorMessage)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterTooManyIterations(): Unit = {
|
||||||
def testAlterTooManyIterations(quorum: String): Unit = {
|
|
||||||
val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1)
|
val upsertionTooFewIterations = new AlterUserScramCredentialsRequestData.ScramCredentialUpsertion().setName(user1)
|
||||||
.setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(Integer.MAX_VALUE)
|
.setMechanism(ScramMechanism.SCRAM_SHA_256.`type`).setIterations(Integer.MAX_VALUE)
|
||||||
.setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
.setSalt(saltBytes).setSaltedPassword(saltedPasswordBytes)
|
||||||
|
@ -224,9 +218,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
assertEquals("Too many iterations", results.get(0).errorMessage)
|
assertEquals("Too many iterations", results.get(0).errorMessage)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDeleteSomethingThatDoesNotExist(): Unit = {
|
||||||
def testDeleteSomethingThatDoesNotExist(quorum: String): Unit = {
|
|
||||||
val request = new AlterUserScramCredentialsRequest.Builder(
|
val request = new AlterUserScramCredentialsRequest.Builder(
|
||||||
new AlterUserScramCredentialsRequestData()
|
new AlterUserScramCredentialsRequestData()
|
||||||
.setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
|
.setDeletions(util.Arrays.asList(new AlterUserScramCredentialsRequestData.ScramCredentialDeletion().setName(user1).setMechanism(ScramMechanism.SCRAM_SHA_256.`type`)))
|
||||||
|
@ -238,9 +231,8 @@ class AlterUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
checkAllErrorsAlteringCredentials(results, Errors.RESOURCE_NOT_FOUND, "when deleting a non-existing credential")
|
checkAllErrorsAlteringCredentials(results, Errors.RESOURCE_NOT_FOUND, "when deleting a non-existing credential")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAlterAndDescribe(): Unit = {
|
||||||
def testAlterAndDescribe(quorum: String): Unit = {
|
|
||||||
// create a bunch of credentials
|
// create a bunch of credentials
|
||||||
val request1_0 = new AlterUserScramCredentialsRequest.Builder(
|
val request1_0 = new AlterUserScramCredentialsRequest.Builder(
|
||||||
new AlterUserScramCredentialsRequestData()
|
new AlterUserScramCredentialsRequestData()
|
||||||
|
|
|
@ -49,9 +49,7 @@ import org.apache.kafka.test.{TestUtils => JTestUtils}
|
||||||
import org.junit.jupiter.api.Assertions.assertEquals
|
import org.junit.jupiter.api.Assertions.assertEquals
|
||||||
import org.junit.jupiter.api.Assertions.assertTrue
|
import org.junit.jupiter.api.Assertions.assertTrue
|
||||||
import org.junit.jupiter.api.Assertions.fail
|
import org.junit.jupiter.api.Assertions.fail
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.collection.Seq
|
import scala.collection.Seq
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -126,9 +124,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
waitUserQuota(ThrottledPrincipal.getName, ControllerMutationRate)
|
waitUserQuota(ThrottledPrincipal.getName, ControllerMutationRate)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testSetUnsetQuota(): Unit = {
|
||||||
def testSetUnsetQuota(quorum: String): Unit = {
|
|
||||||
val rate = 1.5
|
val rate = 1.5
|
||||||
val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "User")
|
val principal = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "User")
|
||||||
// Default Value
|
// Default Value
|
||||||
|
@ -143,9 +140,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
waitUserQuota(principal.getName, Long.MaxValue)
|
waitUserQuota(principal.getName, Long.MaxValue)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testQuotaMetric(): Unit = {
|
||||||
def testQuotaMetric(quorum: String): Unit = {
|
|
||||||
asPrincipal(ThrottledPrincipal) {
|
asPrincipal(ThrottledPrincipal) {
|
||||||
// Metric is lazily created
|
// Metric is lazily created
|
||||||
assertTrue(quotaMetric(principal.getName).isEmpty)
|
assertTrue(quotaMetric(principal.getName).isEmpty)
|
||||||
|
@ -166,9 +162,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testStrictCreateTopicsRequest(): Unit = {
|
||||||
def testStrictCreateTopicsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(ThrottledPrincipal) {
|
asPrincipal(ThrottledPrincipal) {
|
||||||
// Create two topics worth of 30 partitions each. As we use a strict quota, we
|
// Create two topics worth of 30 partitions each. As we use a strict quota, we
|
||||||
// expect one to be created and one to be rejected.
|
// expect one to be created and one to be rejected.
|
||||||
|
@ -190,9 +185,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testPermissiveCreateTopicsRequest(): Unit = {
|
||||||
def testPermissiveCreateTopicsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(ThrottledPrincipal) {
|
asPrincipal(ThrottledPrincipal) {
|
||||||
// Create two topics worth of 30 partitions each. As we use a permissive quota, we
|
// Create two topics worth of 30 partitions each. As we use a permissive quota, we
|
||||||
// expect both topics to be created.
|
// expect both topics to be created.
|
||||||
|
@ -204,9 +198,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUnboundedCreateTopicsRequest(): Unit = {
|
||||||
def testUnboundedCreateTopicsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(UnboundedPrincipal) {
|
asPrincipal(UnboundedPrincipal) {
|
||||||
// Create two topics worth of 30 partitions each. As we use an user without quota, we
|
// Create two topics worth of 30 partitions each. As we use an user without quota, we
|
||||||
// expect both topics to be created. The throttle time should be equal to 0.
|
// expect both topics to be created. The throttle time should be equal to 0.
|
||||||
|
@ -216,9 +209,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testStrictDeleteTopicsRequest(): Unit = {
|
||||||
def testStrictDeleteTopicsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(UnboundedPrincipal) {
|
asPrincipal(UnboundedPrincipal) {
|
||||||
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||||
}
|
}
|
||||||
|
@ -244,9 +236,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testPermissiveDeleteTopicsRequest(): Unit = {
|
||||||
def testPermissiveDeleteTopicsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(UnboundedPrincipal) {
|
asPrincipal(UnboundedPrincipal) {
|
||||||
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||||
}
|
}
|
||||||
|
@ -262,9 +253,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUnboundedDeleteTopicsRequest(): Unit = {
|
||||||
def testUnboundedDeleteTopicsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(UnboundedPrincipal) {
|
asPrincipal(UnboundedPrincipal) {
|
||||||
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
createTopics(TopicsWith30Partitions, StrictCreateTopicsRequestVersion)
|
||||||
|
|
||||||
|
@ -276,9 +266,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testStrictCreatePartitionsRequest(): Unit = {
|
||||||
def testStrictCreatePartitionsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(UnboundedPrincipal) {
|
asPrincipal(UnboundedPrincipal) {
|
||||||
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
||||||
}
|
}
|
||||||
|
@ -304,9 +293,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testPermissiveCreatePartitionsRequest(): Unit = {
|
||||||
def testPermissiveCreatePartitionsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(UnboundedPrincipal) {
|
asPrincipal(UnboundedPrincipal) {
|
||||||
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
||||||
}
|
}
|
||||||
|
@ -322,9 +310,8 @@ class ControllerMutationQuotaTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUnboundedCreatePartitionsRequest(): Unit = {
|
||||||
def testUnboundedCreatePartitionsRequest(quorum: String): Unit = {
|
|
||||||
asPrincipal(UnboundedPrincipal) {
|
asPrincipal(UnboundedPrincipal) {
|
||||||
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
createTopics(TopicsWithOnePartition, StrictCreatePartitionsRequestVersion)
|
||||||
|
|
||||||
|
|
|
@ -24,16 +24,14 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCol
|
||||||
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
import org.apache.kafka.common.protocol.{ApiKeys, Errors}
|
||||||
import org.apache.kafka.common.requests.CreateTopicsRequest
|
import org.apache.kafka.common.requests.CreateTopicsRequest
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
|
class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testValidCreateTopicsRequests(): Unit = {
|
||||||
def testValidCreateTopicsRequests(quorum: String): Unit = {
|
|
||||||
// Generated assignments
|
// Generated assignments
|
||||||
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1"))))
|
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1"))))
|
||||||
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic2", replicationFactor = 3))))
|
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic2", replicationFactor = 3))))
|
||||||
|
@ -61,9 +59,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
|
||||||
topicReq("topic14", replicationFactor = -1, numPartitions = 2))))
|
topicReq("topic14", replicationFactor = -1, numPartitions = 2))))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testErrorCreateTopicsRequests(): Unit = {
|
||||||
def testErrorCreateTopicsRequests(quorum: String): Unit = {
|
|
||||||
val existingTopic = "existing-topic"
|
val existingTopic = "existing-topic"
|
||||||
createTopic(existingTopic)
|
createTopic(existingTopic)
|
||||||
// Basic
|
// Basic
|
||||||
|
@ -99,9 +96,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
|
||||||
validateTopicExists("partial-none")
|
validateTopicExists("partial-none")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testInvalidCreateTopicsRequests(): Unit = {
|
||||||
def testInvalidCreateTopicsRequests(quorum: String): Unit = {
|
|
||||||
// Partitions/ReplicationFactor and ReplicaAssignment
|
// Partitions/ReplicationFactor and ReplicaAssignment
|
||||||
validateErrorCreateTopicsRequests(topicsReq(Seq(
|
validateErrorCreateTopicsRequests(topicsReq(Seq(
|
||||||
topicReq("bad-args-topic", numPartitions = 10, replicationFactor = 3,
|
topicReq("bad-args-topic", numPartitions = 10, replicationFactor = 3,
|
||||||
|
@ -114,9 +110,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
|
||||||
Map("bad-args-topic" -> error(Errors.INVALID_REQUEST)), checkErrorMessage = false)
|
Map("bad-args-topic" -> error(Errors.INVALID_REQUEST)), checkErrorMessage = false)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateTopicsRequestVersions(): Unit = {
|
||||||
def testCreateTopicsRequestVersions(quorum: String): Unit = {
|
|
||||||
for (version <- ApiKeys.CREATE_TOPICS.oldestVersion to ApiKeys.CREATE_TOPICS.latestVersion) {
|
for (version <- ApiKeys.CREATE_TOPICS.oldestVersion to ApiKeys.CREATE_TOPICS.latestVersion) {
|
||||||
val topic = s"topic_$version"
|
val topic = s"topic_$version"
|
||||||
val data = new CreateTopicsRequestData()
|
val data = new CreateTopicsRequestData()
|
||||||
|
@ -153,9 +148,8 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateClusterMetadataTopic(): Unit = {
|
||||||
def testCreateClusterMetadataTopic(quorum: String): Unit = {
|
|
||||||
validateErrorCreateTopicsRequests(
|
validateErrorCreateTopicsRequests(
|
||||||
topicsReq(Seq(topicReq(Topic.CLUSTER_METADATA_TOPIC_NAME))),
|
topicsReq(Seq(topicReq(Topic.CLUSTER_METADATA_TOPIC_NAME))),
|
||||||
Map(Topic.CLUSTER_METADATA_TOPIC_NAME ->
|
Map(Topic.CLUSTER_METADATA_TOPIC_NAME ->
|
||||||
|
|
|
@ -26,9 +26,7 @@ import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG
|
import org.apache.kafka.server.config.ServerLogConfigs.CREATE_TOPIC_POLICY_CLASS_NAME_CONFIG
|
||||||
import org.apache.kafka.server.policy.CreateTopicPolicy
|
import org.apache.kafka.server.policy.CreateTopicPolicy
|
||||||
import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata
|
import org.apache.kafka.server.policy.CreateTopicPolicy.RequestMetadata
|
||||||
import org.junit.jupiter.api.TestInfo
|
import org.junit.jupiter.api.{Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
|
@ -46,9 +44,8 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest
|
||||||
Seq(properties)
|
Seq(properties)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testValidCreateTopicsRequests(): Unit = {
|
||||||
def testValidCreateTopicsRequests(quorum: String): Unit = {
|
|
||||||
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1",
|
validateValidCreateTopicsRequests(topicsReq(Seq(topicReq("topic1",
|
||||||
numPartitions = 5))))
|
numPartitions = 5))))
|
||||||
|
|
||||||
|
@ -65,9 +62,8 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest
|
||||||
assignment = Map(0 -> List(1, 0), 1 -> List(0, 1))))))
|
assignment = Map(0 -> List(1, 0), 1 -> List(0, 1))))))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testErrorCreateTopicsRequests(): Unit = {
|
||||||
def testErrorCreateTopicsRequests(quorum: String): Unit = {
|
|
||||||
val existingTopic = "existing-topic"
|
val existingTopic = "existing-topic"
|
||||||
createTopic(existingTopic, 5)
|
createTopic(existingTopic, 5)
|
||||||
|
|
||||||
|
|
|
@ -21,10 +21,8 @@ import kafka.security.JaasTestUtils
|
||||||
import java.util
|
import java.util
|
||||||
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||||
import org.apache.kafka.common.errors.UnsupportedByAuthenticationException
|
import org.apache.kafka.common.errors.UnsupportedByAuthenticationException
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.api.Assertions.assertThrows
|
import org.junit.jupiter.api.Assertions.assertThrows
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.concurrent.ExecutionException
|
import scala.concurrent.ExecutionException
|
||||||
import scala.jdk.javaapi.OptionConverters
|
import scala.jdk.javaapi.OptionConverters
|
||||||
|
@ -48,9 +46,8 @@ class DelegationTokenRequestsOnPlainTextTest extends BaseRequestTest {
|
||||||
config
|
config
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDelegationTokenRequests(): Unit = {
|
||||||
def testDelegationTokenRequests(quorum: String): Unit = {
|
|
||||||
adminClient = Admin.create(createAdminConfig)
|
adminClient = Admin.create(createAdminConfig)
|
||||||
|
|
||||||
val createResult = adminClient.createDelegationToken()
|
val createResult = adminClient.createDelegationToken()
|
||||||
|
|
|
@ -25,9 +25,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||||
import org.apache.kafka.common.utils.SecurityUtils
|
import org.apache.kafka.common.utils.SecurityUtils
|
||||||
import org.apache.kafka.server.config.DelegationTokenManagerConfigs
|
import org.apache.kafka.server.config.DelegationTokenManagerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
import scala.concurrent.ExecutionException
|
import scala.concurrent.ExecutionException
|
||||||
|
@ -65,9 +63,8 @@ class DelegationTokenRequestsTest extends IntegrationTestHarness with SaslSetup
|
||||||
config
|
config
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDelegationTokenRequests(): Unit = {
|
||||||
def testDelegationTokenRequests(quorum: String): Unit = {
|
|
||||||
adminClient = Admin.create(createAdminConfig)
|
adminClient = Admin.create(createAdminConfig)
|
||||||
|
|
||||||
// create token1 with renewer1
|
// create token1 with renewer1
|
||||||
|
|
|
@ -22,9 +22,7 @@ import org.apache.kafka.clients.admin.{Admin, AdminClientConfig}
|
||||||
import org.apache.kafka.common.errors.DelegationTokenDisabledException
|
import org.apache.kafka.common.errors.DelegationTokenDisabledException
|
||||||
import org.apache.kafka.common.security.auth.SecurityProtocol
|
import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||||
import org.junit.jupiter.api.Assertions.assertThrows
|
import org.junit.jupiter.api.Assertions.assertThrows
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
import scala.concurrent.ExecutionException
|
import scala.concurrent.ExecutionException
|
||||||
|
@ -55,9 +53,8 @@ class DelegationTokenRequestsWithDisableTokenFeatureTest extends BaseRequestTest
|
||||||
config
|
config
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDelegationTokenRequests(): Unit = {
|
||||||
def testDelegationTokenRequests(quorum: String): Unit = {
|
|
||||||
adminClient = Admin.create(createAdminConfig)
|
adminClient = Admin.create(createAdminConfig)
|
||||||
|
|
||||||
val createResult = adminClient.createDelegationToken()
|
val createResult = adminClient.createDelegationToken()
|
||||||
|
|
|
@ -25,8 +25,7 @@ import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.requests.{DeleteRecordsRequest, DeleteRecordsResponse}
|
import org.apache.kafka.common.requests.{DeleteRecordsRequest, DeleteRecordsResponse}
|
||||||
import org.apache.kafka.common.serialization.StringSerializer
|
import org.apache.kafka.common.serialization.StringSerializer
|
||||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.Collections
|
import java.util.Collections
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
|
@ -36,9 +35,8 @@ class DeleteRecordsRequestTest extends BaseRequestTest {
|
||||||
private val TIMEOUT_MS = 1000
|
private val TIMEOUT_MS = 1000
|
||||||
private val MESSAGES_PRODUCED_PER_PARTITION = 10
|
private val MESSAGES_PRODUCED_PER_PARTITION = 10
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDeleteRecordsHappyCase(): Unit = {
|
||||||
def testDeleteRecordsHappyCase(quorum: String): Unit = {
|
|
||||||
val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords
|
val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords
|
||||||
|
|
||||||
// Create the DeleteRecord request requesting deletion of offset which is not present
|
// Create the DeleteRecord request requesting deletion of offset which is not present
|
||||||
|
@ -61,9 +59,8 @@ class DeleteRecordsRequestTest extends BaseRequestTest {
|
||||||
validateLogStartOffsetForTopic(topicPartition, offsetToDelete)
|
validateLogStartOffsetForTopic(topicPartition, offsetToDelete)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testErrorWhenDeletingRecordsWithInvalidOffset(): Unit = {
|
||||||
def testErrorWhenDeletingRecordsWithInvalidOffset(quorum: String): Unit = {
|
|
||||||
val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords
|
val (topicPartition: TopicPartition, leaderId: Int) = createTopicAndSendRecords
|
||||||
|
|
||||||
// Create the DeleteRecord request requesting deletion of offset which is not present
|
// Create the DeleteRecord request requesting deletion of offset which is not present
|
||||||
|
@ -86,9 +83,8 @@ class DeleteRecordsRequestTest extends BaseRequestTest {
|
||||||
validateLogStartOffsetForTopic(topicPartition, 0)
|
validateLogStartOffsetForTopic(topicPartition, 0)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testErrorWhenDeletingRecordsWithInvalidTopic(): Unit = {
|
||||||
def testErrorWhenDeletingRecordsWithInvalidTopic(quorum: String): Unit = {
|
|
||||||
val invalidTopicPartition = new TopicPartition("invalid-topic", 0)
|
val invalidTopicPartition = new TopicPartition("invalid-topic", 0)
|
||||||
// Create the DeleteRecord request requesting deletion of offset which is not present
|
// Create the DeleteRecord request requesting deletion of offset which is not present
|
||||||
val offsetToDelete = 1
|
val offsetToDelete = 1
|
||||||
|
|
|
@ -28,17 +28,15 @@ import org.apache.kafka.common.requests.DeleteTopicsResponse
|
||||||
import org.apache.kafka.common.requests.MetadataRequest
|
import org.apache.kafka.common.requests.MetadataRequest
|
||||||
import org.apache.kafka.common.requests.MetadataResponse
|
import org.apache.kafka.common.requests.MetadataResponse
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.collection.Seq
|
import scala.collection.Seq
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
|
class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testTopicDeletionClusterHasOfflinePartitions(): Unit = {
|
||||||
def testTopicDeletionClusterHasOfflinePartitions(quorum: String): Unit = {
|
|
||||||
// Create two topics with one partition/replica. Make one of them offline.
|
// Create two topics with one partition/replica. Make one of them offline.
|
||||||
val offlineTopic = "topic-1"
|
val offlineTopic = "topic-1"
|
||||||
val onlineTopic = "topic-2"
|
val onlineTopic = "topic-2"
|
||||||
|
@ -70,9 +68,8 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
|
||||||
"The topics are found in the Broker's cache")
|
"The topics are found in the Broker's cache")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testValidDeleteTopicRequests(): Unit = {
|
||||||
def testValidDeleteTopicRequests(quorum: String): Unit = {
|
|
||||||
val timeout = 10000
|
val timeout = 10000
|
||||||
// Single topic
|
// Single topic
|
||||||
createTopic("topic-1")
|
createTopic("topic-1")
|
||||||
|
@ -138,9 +135,8 @@ class DeleteTopicsRequestTest extends BaseRequestTest with Logging {
|
||||||
connectAndReceive[DeleteTopicsResponse](request, destination = socketServer)
|
connectAndReceive[DeleteTopicsResponse](request, destination = socketServer)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDeleteTopicsVersions(): Unit = {
|
||||||
def testDeleteTopicsVersions(quorum: String): Unit = {
|
|
||||||
val timeout = 10000
|
val timeout = 10000
|
||||||
for (version <- ApiKeys.DELETE_TOPICS.oldestVersion to ApiKeys.DELETE_TOPICS.latestVersion) {
|
for (version <- ApiKeys.DELETE_TOPICS.oldestVersion to ApiKeys.DELETE_TOPICS.latestVersion) {
|
||||||
info(s"Creating and deleting tests for version $version")
|
info(s"Creating and deleting tests for version $version")
|
||||||
|
|
|
@ -25,9 +25,7 @@ import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse}
|
import org.apache.kafka.common.requests.{DeleteTopicsRequest, DeleteTopicsResponse}
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.TestInfo
|
import org.junit.jupiter.api.{Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest {
|
class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest {
|
||||||
|
|
||||||
|
@ -48,9 +46,8 @@ class DeleteTopicsRequestWithDeletionDisabledTest extends BaseRequestTest {
|
||||||
props.map(KafkaConfig.fromProps)
|
props.map(KafkaConfig.fromProps)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDeleteRecordsRequest(): Unit = {
|
||||||
def testDeleteRecordsRequest(quorum: String): Unit = {
|
|
||||||
val topic = "topic-1"
|
val topic = "topic-1"
|
||||||
val request = new DeleteTopicsRequest.Builder(
|
val request = new DeleteTopicsRequest.Builder(
|
||||||
new DeleteTopicsRequestData()
|
new DeleteTopicsRequestData()
|
||||||
|
|
|
@ -27,9 +27,7 @@ import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
|
||||||
import org.apache.kafka.security.authorizer.AclEntry
|
import org.apache.kafka.security.authorizer.AclEntry
|
||||||
import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs}
|
import org.apache.kafka.server.config.{ServerConfigs, ReplicationConfigs}
|
||||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.lang.{Byte => JByte}
|
import java.lang.{Byte => JByte}
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
@ -48,15 +46,13 @@ class DescribeClusterRequestTest extends BaseRequestTest {
|
||||||
doSetup(testInfo, createOffsetsTopic = false)
|
doSetup(testInfo, createOffsetsTopic = false)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeClusterRequestIncludingClusterAuthorizedOperations(): Unit = {
|
||||||
def testDescribeClusterRequestIncludingClusterAuthorizedOperations(quorum: String): Unit = {
|
|
||||||
testDescribeClusterRequest(true)
|
testDescribeClusterRequest(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeClusterRequestExcludingClusterAuthorizedOperations(): Unit = {
|
||||||
def testDescribeClusterRequestExcludingClusterAuthorizedOperations(quorum: String): Unit = {
|
|
||||||
testDescribeClusterRequest(false)
|
testDescribeClusterRequest(false)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -25,8 +25,7 @@ import org.apache.kafka.common.message.DescribeLogDirsRequestData
|
||||||
import org.apache.kafka.common.protocol.Errors
|
import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.requests._
|
import org.apache.kafka.common.requests._
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
|
@ -39,9 +38,8 @@ class DescribeLogDirsRequestTest extends BaseRequestTest {
|
||||||
val tp0 = new TopicPartition(topic, 0)
|
val tp0 = new TopicPartition(topic, 0)
|
||||||
val tp1 = new TopicPartition(topic, 1)
|
val tp1 = new TopicPartition(topic, 1)
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeLogDirsRequest(): Unit = {
|
||||||
def testDescribeLogDirsRequest(quorum: String): Unit = {
|
|
||||||
val onlineDir = new File(brokers.head.config.logDirs.head).getAbsolutePath
|
val onlineDir = new File(brokers.head.config.logDirs.head).getAbsolutePath
|
||||||
val offlineDir = new File(brokers.head.config.logDirs.tail.head).getAbsolutePath
|
val offlineDir = new File(brokers.head.config.logDirs.tail.head).getAbsolutePath
|
||||||
brokers.head.replicaManager.handleLogDirFailure(offlineDir)
|
brokers.head.replicaManager.handleLogDirFailure(offlineDir)
|
||||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.kafka.common.requests.{DescribeUserScramCredentialsRequest, De
|
||||||
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
|
import org.apache.kafka.metadata.authorizer.StandardAuthorizer
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
|
||||||
|
@ -39,9 +38,8 @@ class DescribeUserScramCredentialsRequestNotAuthorizedTest extends BaseRequestTe
|
||||||
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName)
|
properties.put(BrokerSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, classOf[DescribeCredentialsTest.TestPrincipalBuilderReturningUnauthorized].getName)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeNotAuthorized(): Unit = {
|
||||||
def testDescribeNotAuthorized(quorum: String): Unit = {
|
|
||||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||||
new DescribeUserScramCredentialsRequestData()).build()
|
new DescribeUserScramCredentialsRequestData()).build()
|
||||||
val response = sendDescribeUserScramCredentialsRequest(request)
|
val response = sendDescribeUserScramCredentialsRequest(request)
|
||||||
|
|
|
@ -28,8 +28,6 @@ import org.apache.kafka.common.security.authenticator.DefaultKafkaPrincipalBuild
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
|
@ -48,9 +46,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
super.setUp(testInfo)
|
super.setUp(testInfo)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeNothing(): Unit = {
|
||||||
def testDescribeNothing(quorum: String): Unit = {
|
|
||||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||||
new DescribeUserScramCredentialsRequestData()).build()
|
new DescribeUserScramCredentialsRequestData()).build()
|
||||||
val response = sendDescribeUserScramCredentialsRequest(request)
|
val response = sendDescribeUserScramCredentialsRequest(request)
|
||||||
|
@ -60,9 +57,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
assertEquals(0, response.data.results.size, "Expected no credentials when describing everything and there are no credentials")
|
assertEquals(0, response.data.results.size, "Expected no credentials when describing everything and there are no credentials")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeWithNull(): Unit = {
|
||||||
def testDescribeWithNull(quorum: String): Unit = {
|
|
||||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||||
new DescribeUserScramCredentialsRequestData().setUsers(null)).build()
|
new DescribeUserScramCredentialsRequestData().setUsers(null)).build()
|
||||||
val response = sendDescribeUserScramCredentialsRequest(request)
|
val response = sendDescribeUserScramCredentialsRequest(request)
|
||||||
|
@ -82,9 +78,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
assertEquals(Errors.NONE.code, error, "Did not expect controller error when routed to non-controller")
|
assertEquals(Errors.NONE.code, error, "Did not expect controller error when routed to non-controller")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDescribeSameUserTwice(): Unit = {
|
||||||
def testDescribeSameUserTwice(quorum: String): Unit = {
|
|
||||||
val user = "user1"
|
val user = "user1"
|
||||||
val userName = new UserName().setName(user)
|
val userName = new UserName().setName(user)
|
||||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||||
|
@ -98,9 +93,8 @@ class DescribeUserScramCredentialsRequestTest extends BaseRequestTest {
|
||||||
assertEquals(s"Cannot describe SCRAM credentials for the same user twice in a single request: $user", result.errorMessage)
|
assertEquals(s"Cannot describe SCRAM credentials for the same user twice in a single request: $user", result.errorMessage)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUnknownUser(): Unit = {
|
||||||
def testUnknownUser(quorum: String): Unit = {
|
|
||||||
val unknownUser = "unknownUser"
|
val unknownUser = "unknownUser"
|
||||||
val request = new DescribeUserScramCredentialsRequest.Builder(
|
val request = new DescribeUserScramCredentialsRequest.Builder(
|
||||||
new DescribeUserScramCredentialsRequestData().setUsers(List(new UserName().setName(unknownUser)).asJava)).build()
|
new DescribeUserScramCredentialsRequestData().setUsers(List(new UserName().setName(unknownUser)).asJava)).build()
|
||||||
|
|
|
@ -40,8 +40,6 @@ import org.apache.kafka.storage.internals.log.{LogConfig, UnifiedLog}
|
||||||
import org.apache.kafka.test.TestUtils.assertFutureThrows
|
import org.apache.kafka.test.TestUtils.assertFutureThrows
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{Test, Timeout}
|
import org.junit.jupiter.api.{Test, Timeout}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
import org.mockito.ArgumentCaptor
|
import org.mockito.ArgumentCaptor
|
||||||
import org.mockito.ArgumentMatchers.any
|
import org.mockito.ArgumentMatchers.any
|
||||||
import org.mockito.Mockito._
|
import org.mockito.Mockito._
|
||||||
|
@ -61,9 +59,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
List(KafkaConfig.fromProps(cfg))
|
List(KafkaConfig.fromProps(cfg))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testConfigChange(): Unit = {
|
||||||
def testConfigChange(quorum: String): Unit = {
|
|
||||||
val oldVal: java.lang.Long = 100000L
|
val oldVal: java.lang.Long = 100000L
|
||||||
val newVal: java.lang.Long = 200000L
|
val newVal: java.lang.Long = 200000L
|
||||||
val tp = new TopicPartition("test", 0)
|
val tp = new TopicPartition("test", 0)
|
||||||
|
@ -95,9 +92,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDynamicTopicConfigChange(): Unit = {
|
||||||
def testDynamicTopicConfigChange(quorum: String): Unit = {
|
|
||||||
val tp = new TopicPartition("test", 0)
|
val tp = new TopicPartition("test", 0)
|
||||||
val oldSegmentSize = 1000
|
val oldSegmentSize = 1000
|
||||||
val logProps = new Properties()
|
val logProps = new Properties()
|
||||||
|
@ -180,59 +176,52 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testClientIdQuotaConfigChange(): Unit = {
|
||||||
def testClientIdQuotaConfigChange(quorum: String): Unit = {
|
|
||||||
val m = new util.HashMap[String, String]
|
val m = new util.HashMap[String, String]
|
||||||
m.put(CLIENT_ID, "testClient")
|
m.put(CLIENT_ID, "testClient")
|
||||||
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUserQuotaConfigChange(): Unit = {
|
||||||
def testUserQuotaConfigChange(quorum: String): Unit = {
|
|
||||||
val m = new util.HashMap[String, String]
|
val m = new util.HashMap[String, String]
|
||||||
m.put(USER, "ANONYMOUS")
|
m.put(USER, "ANONYMOUS")
|
||||||
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUserClientIdQuotaChange(): Unit = {
|
||||||
def testUserClientIdQuotaChange(quorum: String): Unit = {
|
|
||||||
val m = new util.HashMap[String, String]
|
val m = new util.HashMap[String, String]
|
||||||
m.put(USER, "ANONYMOUS")
|
m.put(USER, "ANONYMOUS")
|
||||||
m.put(CLIENT_ID, "testClient")
|
m.put(CLIENT_ID, "testClient")
|
||||||
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDefaultClientIdQuotaConfigChange(): Unit = {
|
||||||
def testDefaultClientIdQuotaConfigChange(quorum: String): Unit = {
|
|
||||||
val m = new util.HashMap[String, String]
|
val m = new util.HashMap[String, String]
|
||||||
m.put(CLIENT_ID, null)
|
m.put(CLIENT_ID, null)
|
||||||
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDefaultUserQuotaConfigChange(): Unit = {
|
||||||
def testDefaultUserQuotaConfigChange(quorum: String): Unit = {
|
|
||||||
val m = new util.HashMap[String, String]
|
val m = new util.HashMap[String, String]
|
||||||
m.put(USER, null)
|
m.put(USER, null)
|
||||||
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDefaultUserClientIdQuotaConfigChange(): Unit = {
|
||||||
def testDefaultUserClientIdQuotaConfigChange(quorum: String): Unit = {
|
|
||||||
val m = new util.HashMap[String, String]
|
val m = new util.HashMap[String, String]
|
||||||
m.put(USER, null)
|
m.put(USER, null)
|
||||||
m.put(CLIENT_ID, null)
|
m.put(CLIENT_ID, null)
|
||||||
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
testQuotaConfigChange(new ClientQuotaEntity(m), KafkaPrincipal.ANONYMOUS, "testClient")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIpQuotaInitialization(): Unit = {
|
||||||
def testIpQuotaInitialization(quorum: String): Unit = {
|
|
||||||
val broker = brokers.head
|
val broker = brokers.head
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
try {
|
try {
|
||||||
|
@ -252,9 +241,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIpQuotaConfigChange(): Unit = {
|
||||||
def testIpQuotaConfigChange(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
try {
|
try {
|
||||||
val alterations = util.Arrays.asList(
|
val alterations = util.Arrays.asList(
|
||||||
|
@ -296,9 +284,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
|
|
||||||
private def tempTopic() : String = "testTopic" + random.nextInt(1000000)
|
private def tempTopic() : String = "testTopic" + random.nextInt(1000000)
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testConfigChangeOnNonExistingTopicWithAdminClient(): Unit = {
|
||||||
def testConfigChangeOnNonExistingTopicWithAdminClient(quorum: String): Unit = {
|
|
||||||
val topic = tempTopic()
|
val topic = tempTopic()
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
try {
|
try {
|
||||||
|
@ -314,9 +301,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIncrementalAlterDefaultTopicConfig(): Unit = {
|
||||||
def testIncrementalAlterDefaultTopicConfig(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
try {
|
try {
|
||||||
val resource = new ConfigResource(ConfigResource.Type.TOPIC, "")
|
val resource = new ConfigResource(ConfigResource.Type.TOPIC, "")
|
||||||
|
@ -346,9 +332,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testBrokerIdConfigChangeAndDelete(): Unit = {
|
||||||
def testBrokerIdConfigChangeAndDelete(quorum: String): Unit = {
|
|
||||||
val newValue: Long = 100000L
|
val newValue: Long = 100000L
|
||||||
val brokerId: String = this.brokers.head.config.brokerId.toString
|
val brokerId: String = this.brokers.head.config.brokerId.toString
|
||||||
setBrokerConfigs(brokerId, newValue)
|
setBrokerConfigs(brokerId, newValue)
|
||||||
|
@ -370,9 +355,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDefaultBrokerIdConfigChangeAndDelete(): Unit = {
|
||||||
def testDefaultBrokerIdConfigChangeAndDelete(quorum: String): Unit = {
|
|
||||||
val newValue: Long = 100000L
|
val newValue: Long = 100000L
|
||||||
val brokerId: String = ""
|
val brokerId: String = ""
|
||||||
setBrokerConfigs(brokerId, newValue)
|
setBrokerConfigs(brokerId, newValue)
|
||||||
|
@ -393,9 +377,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDefaultAndBrokerIdConfigChange(): Unit = {
|
||||||
def testDefaultAndBrokerIdConfigChange(quorum: String): Unit = {
|
|
||||||
val newValue: Long = 100000L
|
val newValue: Long = 100000L
|
||||||
val brokerId: String = this.brokers.head.config.brokerId.toString
|
val brokerId: String = this.brokers.head.config.brokerId.toString
|
||||||
setBrokerConfigs(brokerId, newValue)
|
setBrokerConfigs(brokerId, newValue)
|
||||||
|
@ -411,9 +394,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testDynamicGroupConfigChange(): Unit = {
|
||||||
def testDynamicGroupConfigChange(quorum: String): Unit = {
|
|
||||||
val newSessionTimeoutMs = 50000
|
val newSessionTimeoutMs = 50000
|
||||||
val consumerGroupId = "group-foo"
|
val consumerGroupId = "group-foo"
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
|
@ -438,9 +420,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
assertEquals(newSessionTimeoutMs, groupConfig.consumerSessionTimeoutMs())
|
assertEquals(newSessionTimeoutMs, groupConfig.consumerSessionTimeoutMs())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft+kip848"))
|
def testDynamicShareGroupConfigChange(): Unit = {
|
||||||
def testDynamicShareGroupConfigChange(quorum: String): Unit = {
|
|
||||||
val newRecordLockDurationMs = 50000
|
val newRecordLockDurationMs = 50000
|
||||||
val shareGroupId = "group-foo"
|
val shareGroupId = "group-foo"
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
|
@ -465,9 +446,8 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
|
||||||
assertEquals(newRecordLockDurationMs, groupConfig.shareRecordLockDurationMs)
|
assertEquals(newRecordLockDurationMs, groupConfig.shareRecordLockDurationMs)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIncrementalAlterDefaultGroupConfig(): Unit = {
|
||||||
def testIncrementalAlterDefaultGroupConfig(quorum: String): Unit = {
|
|
||||||
val admin = createAdminClient()
|
val admin = createAdminClient()
|
||||||
try {
|
try {
|
||||||
val resource = new ConfigResource(ConfigResource.Type.GROUP, "")
|
val resource = new ConfigResource(ConfigResource.Type.GROUP, "")
|
||||||
|
|
|
@ -36,8 +36,7 @@ import org.apache.kafka.common.utils.ByteUtils
|
||||||
import org.apache.kafka.common.{TopicPartition, Uuid, requests}
|
import org.apache.kafka.common.{TopicPartition, Uuid, requests}
|
||||||
import org.apache.kafka.server.config.ServerLogConfigs
|
import org.apache.kafka.server.config.ServerLogConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
|
@ -118,9 +117,8 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testProduceRequestWithNullClientId(): Unit = {
|
||||||
def testProduceRequestWithNullClientId(quorum: String): Unit = {
|
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val correlationId = -1
|
val correlationId = -1
|
||||||
|
@ -166,27 +164,23 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
|
||||||
assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode), "There should be no error")
|
assertEquals(Errors.NONE, Errors.forCode(partitionProduceResponse.errorCode), "There should be no error")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testHeaderOnlyRequest(): Unit = {
|
||||||
def testHeaderOnlyRequest(quorum: String): Unit = {
|
|
||||||
verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, 1))
|
verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, 1))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testInvalidApiKeyRequest(): Unit = {
|
||||||
def testInvalidApiKeyRequest(quorum: String): Unit = {
|
|
||||||
verifyDisconnect(requestHeaderBytes(-1, 0))
|
verifyDisconnect(requestHeaderBytes(-1, 0))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testInvalidApiVersionRequest(): Unit = {
|
||||||
def testInvalidApiVersionRequest(quorum: String): Unit = {
|
|
||||||
verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, -1))
|
verifyDisconnect(requestHeaderBytes(ApiKeys.PRODUCE.id, -1))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testMalformedHeaderRequest(): Unit = {
|
||||||
def testMalformedHeaderRequest(quorum: String): Unit = {
|
|
||||||
val serializedBytes = {
|
val serializedBytes = {
|
||||||
// Only send apiKey and apiVersion
|
// Only send apiKey and apiVersion
|
||||||
val buffer = ByteBuffer.allocate(
|
val buffer = ByteBuffer.allocate(
|
||||||
|
|
|
@ -25,9 +25,7 @@ import org.apache.kafka.common.requests.FetchRequest.PartitionData
|
||||||
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
|
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.{Optional, Properties}
|
import java.util.{Optional, Properties}
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -104,9 +102,8 @@ class FetchRequestMaxBytesTest extends BaseRequestTest {
|
||||||
* Note that when a single batch is larger than FetchMaxBytes, it will be
|
* Note that when a single batch is larger than FetchMaxBytes, it will be
|
||||||
* returned in full even if this is larger than FetchMaxBytes. See KIP-74.
|
* returned in full even if this is larger than FetchMaxBytes. See KIP-74.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testConsumeMultipleRecords(): Unit = {
|
||||||
def testConsumeMultipleRecords(quorum: String): Unit = {
|
|
||||||
createTopics()
|
createTopics()
|
||||||
|
|
||||||
expectNextRecords(IndexedSeq(messages(0), messages(1)), 0)
|
expectNextRecords(IndexedSeq(messages(0), messages(1)), 0)
|
||||||
|
|
|
@ -26,8 +26,7 @@ import org.apache.kafka.common.serialization.StringSerializer
|
||||||
import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition, Uuid}
|
import org.apache.kafka.common.{IsolationLevel, TopicIdPartition, TopicPartition, Uuid}
|
||||||
import org.apache.kafka.server.record.BrokerCompressionType
|
import org.apache.kafka.server.record.BrokerCompressionType
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util
|
import java.util
|
||||||
import java.util.Optional
|
import java.util.Optional
|
||||||
|
@ -41,9 +40,8 @@ import scala.util.Random
|
||||||
*/
|
*/
|
||||||
class FetchRequestTest extends BaseFetchRequestTest {
|
class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testBrokerRespectsPartitionsOrderAndSizeLimits(): Unit = {
|
||||||
def testBrokerRespectsPartitionsOrderAndSizeLimits(quorum: String): Unit = {
|
|
||||||
initProducer()
|
initProducer()
|
||||||
|
|
||||||
val messagesPerPartition = 9
|
val messagesPerPartition = 9
|
||||||
|
@ -144,9 +142,8 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
evaluateResponse4(fetchResponse4V12, 12)
|
evaluateResponse4(fetchResponse4V12, 12)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testFetchRequestV4WithReadCommitted(): Unit = {
|
||||||
def testFetchRequestV4WithReadCommitted(quorum: String): Unit = {
|
|
||||||
initProducer()
|
initProducer()
|
||||||
val maxPartitionBytes = 200
|
val maxPartitionBytes = 200
|
||||||
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head
|
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1).head
|
||||||
|
@ -163,9 +160,8 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
assertTrue(records(partitionData).map(_.sizeInBytes).sum > 0)
|
assertTrue(records(partitionData).map(_.sizeInBytes).sum > 0)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testFetchRequestToNonReplica(): Unit = {
|
||||||
def testFetchRequestToNonReplica(quorum: String): Unit = {
|
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val partition = 0
|
val partition = 0
|
||||||
val topicPartition = new TopicPartition(topic, partition)
|
val topicPartition = new TopicPartition(topic, partition)
|
||||||
|
@ -194,15 +190,13 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, oldPartitionData.errorCode)
|
assertEquals(Errors.NOT_LEADER_OR_FOLLOWER.code, oldPartitionData.errorCode)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testLastFetchedEpochValidation(): Unit = {
|
||||||
def testLastFetchedEpochValidation(quorum: String): Unit = {
|
|
||||||
checkLastFetchedEpochValidation(ApiKeys.FETCH.latestVersion())
|
checkLastFetchedEpochValidation(ApiKeys.FETCH.latestVersion())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testLastFetchedEpochValidationV12(): Unit = {
|
||||||
def testLastFetchedEpochValidationV12(quorum: String): Unit = {
|
|
||||||
checkLastFetchedEpochValidation(12)
|
checkLastFetchedEpochValidation(12)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -249,15 +243,13 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
assertEquals(firstEpochEndOffset, divergingEpoch.endOffset)
|
assertEquals(firstEpochEndOffset, divergingEpoch.endOffset)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCurrentEpochValidation(): Unit = {
|
||||||
def testCurrentEpochValidation(quorum: String): Unit = {
|
|
||||||
checkCurrentEpochValidation(ApiKeys.FETCH.latestVersion())
|
checkCurrentEpochValidation(ApiKeys.FETCH.latestVersion())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCurrentEpochValidationV12(): Unit = {
|
||||||
def testCurrentEpochValidationV12(quorum: String): Unit = {
|
|
||||||
checkCurrentEpochValidation(12)
|
checkCurrentEpochValidation(12)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -299,15 +291,13 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch - 1))
|
assertResponseErrorForEpoch(Errors.FENCED_LEADER_EPOCH, followerId, Optional.of(secondLeaderEpoch - 1))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testEpochValidationWithinFetchSession(): Unit = {
|
||||||
def testEpochValidationWithinFetchSession(quorum: String): Unit = {
|
|
||||||
checkEpochValidationWithinFetchSession(ApiKeys.FETCH.latestVersion())
|
checkEpochValidationWithinFetchSession(ApiKeys.FETCH.latestVersion())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testEpochValidationWithinFetchSessionV12(): Unit = {
|
||||||
def testEpochValidationWithinFetchSessionV12(quorum: String): Unit = {
|
|
||||||
checkEpochValidationWithinFetchSession(12)
|
checkEpochValidationWithinFetchSession(12)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -367,9 +357,8 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
* those partitions are returned in all incremental fetch requests.
|
* those partitions are returned in all incremental fetch requests.
|
||||||
* This tests using FetchRequests that don't use topic IDs
|
* This tests using FetchRequests that don't use topic IDs
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCreateIncrementalFetchWithPartitionsInErrorV12(): Unit = {
|
||||||
def testCreateIncrementalFetchWithPartitionsInErrorV12(quorum: String): Unit = {
|
|
||||||
def createConsumerFetchRequest(topicPartitions: Seq[TopicPartition],
|
def createConsumerFetchRequest(topicPartitions: Seq[TopicPartition],
|
||||||
metadata: JFetchMetadata,
|
metadata: JFetchMetadata,
|
||||||
toForget: Seq[TopicIdPartition]): FetchRequest =
|
toForget: Seq[TopicIdPartition]): FetchRequest =
|
||||||
|
@ -430,9 +419,8 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
/**
|
/**
|
||||||
* Test that when a Fetch Request receives an unknown topic ID, it returns a top level error.
|
* Test that when a Fetch Request receives an unknown topic ID, it returns a top level error.
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testFetchWithPartitionsWithIdError(): Unit = {
|
||||||
def testFetchWithPartitionsWithIdError(quorum: String): Unit = {
|
|
||||||
def createConsumerFetchRequest(fetchData: util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData],
|
def createConsumerFetchRequest(fetchData: util.LinkedHashMap[TopicPartition, FetchRequest.PartitionData],
|
||||||
metadata: JFetchMetadata,
|
metadata: JFetchMetadata,
|
||||||
toForget: Seq[TopicIdPartition]): FetchRequest = {
|
toForget: Seq[TopicIdPartition]): FetchRequest = {
|
||||||
|
@ -475,9 +463,8 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
assertEquals(Errors.UNKNOWN_TOPIC_ID.code, responseData1.get(bar0).errorCode)
|
assertEquals(Errors.UNKNOWN_TOPIC_ID.code, responseData1.get(bar0).errorCode)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testZStdCompressedTopic(): Unit = {
|
||||||
def testZStdCompressedTopic(quorum: String): Unit = {
|
|
||||||
// ZSTD compressed topic
|
// ZSTD compressed topic
|
||||||
val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.ZSTD.name)
|
val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.ZSTD.name)
|
||||||
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head
|
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head
|
||||||
|
@ -523,9 +510,8 @@ class FetchRequestTest extends BaseFetchRequestTest {
|
||||||
assertEquals(3, records(data2).size)
|
assertEquals(3, records(data2).size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testZStdCompressedRecords(): Unit = {
|
||||||
def testZStdCompressedRecords(quorum: String): Unit = {
|
|
||||||
// Producer compressed topic
|
// Producer compressed topic
|
||||||
val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.PRODUCER.name)
|
val topicConfig = Map(TopicConfig.COMPRESSION_TYPE_CONFIG -> BrokerCompressionType.PRODUCER.name)
|
||||||
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head
|
val (topicPartition, leaderId) = createTopics(numTopics = 1, numPartitions = 1, configs = topicConfig).head
|
||||||
|
|
|
@ -24,9 +24,7 @@ import org.apache.kafka.common.security.auth.SecurityProtocol
|
||||||
import org.apache.kafka.server.config.QuotaConfig
|
import org.apache.kafka.server.config.QuotaConfig
|
||||||
import org.apache.kafka.server.metrics.MetricConfigs
|
import org.apache.kafka.server.metrics.MetricConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.net.Socket
|
import java.net.Socket
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
@ -63,9 +61,8 @@ class KafkaMetricReporterExceptionHandlingTest extends BaseRequestTest {
|
||||||
super.tearDown()
|
super.tearDown()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testBothReportersAreInvoked(): Unit = {
|
||||||
def testBothReportersAreInvoked(quorum: String): Unit = {
|
|
||||||
val port = anySocketServer.boundPort(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))
|
val port = anySocketServer.boundPort(ListenerName.forSecurityProtocol(SecurityProtocol.PLAINTEXT))
|
||||||
val socket = new Socket("localhost", port)
|
val socket = new Socket("localhost", port)
|
||||||
socket.setSoTimeout(10000)
|
socket.setSoTimeout(10000)
|
||||||
|
|
|
@ -23,10 +23,8 @@ import org.apache.kafka.common.metrics.{KafkaMetric, MetricsContext, MetricsRepo
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.apache.kafka.server.metrics.MetricConfigs
|
import org.apache.kafka.server.metrics.MetricConfigs
|
||||||
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
|
|
||||||
object KafkaMetricsReporterTest {
|
object KafkaMetricsReporterTest {
|
||||||
|
@ -78,9 +76,8 @@ class KafkaMetricsReporterTest extends QuorumTestHarness {
|
||||||
broker.startup()
|
broker.startup()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testMetricsContextNamespacePresent(): Unit = {
|
||||||
def testMetricsContextNamespacePresent(quorum: String): Unit = {
|
|
||||||
assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID.get())
|
assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.CLUSTERID.get())
|
||||||
assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.NODEID.get())
|
assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.NODEID.get())
|
||||||
assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX.get())
|
assertNotNull(KafkaMetricsReporterTest.MockMetricsReporter.JMXPREFIX.get())
|
||||||
|
|
|
@ -24,8 +24,7 @@ import org.apache.kafka.common.requests.{ListOffsetsRequest, ListOffsetsResponse
|
||||||
import org.apache.kafka.common.{IsolationLevel, TopicPartition}
|
import org.apache.kafka.common.{IsolationLevel, TopicPartition}
|
||||||
import org.apache.kafka.server.config.ServerConfigs
|
import org.apache.kafka.server.config.ServerConfigs
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.{Optional, Properties}
|
import java.util.{Optional, Properties}
|
||||||
import scala.collection.Seq
|
import scala.collection.Seq
|
||||||
|
@ -43,9 +42,8 @@ class ListOffsetsRequestTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testListOffsetsErrorCodes(): Unit = {
|
||||||
def testListOffsetsErrorCodes(quorum: String): Unit = {
|
|
||||||
val targetTimes = List(new ListOffsetsTopic()
|
val targetTimes = List(new ListOffsetsTopic()
|
||||||
.setName(topic)
|
.setName(topic)
|
||||||
.setPartitions(List(new ListOffsetsPartition()
|
.setPartitions(List(new ListOffsetsPartition()
|
||||||
|
@ -108,9 +106,8 @@ class ListOffsetsRequestTest extends BaseRequestTest {
|
||||||
assertResponseError(error, brokerId, request)
|
assertResponseError(error, brokerId, request)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCurrentEpochValidation(): Unit = {
|
||||||
def testCurrentEpochValidation(quorum: String): Unit = {
|
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
|
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
|
||||||
|
@ -168,9 +165,8 @@ class ListOffsetsRequestTest extends BaseRequestTest {
|
||||||
(partitionData.offset, partitionData.leaderEpoch, partitionData.errorCode())
|
(partitionData.offset, partitionData.leaderEpoch, partitionData.errorCode())
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testResponseIncludesLeaderEpoch(): Unit = {
|
||||||
def testResponseIncludesLeaderEpoch(quorum: String): Unit = {
|
|
||||||
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
|
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
|
||||||
val firstLeaderId = partitionToLeader(partition.partition)
|
val firstLeaderId = partitionToLeader(partition.partition)
|
||||||
|
|
||||||
|
@ -209,9 +205,8 @@ class ListOffsetsRequestTest extends BaseRequestTest {
|
||||||
assertEquals((9L, firstLeaderEpoch, Errors.NONE.code), fetchOffsetAndEpochWithError(secondLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, -1))
|
assertEquals((9L, firstLeaderEpoch, Errors.NONE.code), fetchOffsetAndEpochWithError(secondLeaderId, ListOffsetsRequest.MAX_TIMESTAMP, -1))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testResponseDefaultOffsetAndLeaderEpochForAllVersions(): Unit = {
|
||||||
def testResponseDefaultOffsetAndLeaderEpochForAllVersions(quorum: String): Unit = {
|
|
||||||
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
|
val partitionToLeader = createTopic(numPartitions = 1, replicationFactor = 3)
|
||||||
val firstLeaderId = partitionToLeader(partition.partition)
|
val firstLeaderId = partitionToLeader(partition.partition)
|
||||||
|
|
||||||
|
|
|
@ -26,9 +26,7 @@ import org.apache.kafka.common.requests.{FetchRequest, FetchResponse, ListOffset
|
||||||
import org.apache.kafka.common.{IsolationLevel, TopicPartition}
|
import org.apache.kafka.common.{IsolationLevel, TopicPartition}
|
||||||
import org.apache.kafka.storage.internals.log.{LogStartOffsetIncrementReason, OffsetResultHolder, UnifiedLog}
|
import org.apache.kafka.storage.internals.log.{LogStartOffsetIncrementReason, OffsetResultHolder, UnifiedLog}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.Timeout
|
import org.junit.jupiter.api.{Test, Timeout}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.io.File
|
import java.io.File
|
||||||
import java.util.{Optional, Properties, Random}
|
import java.util.{Optional, Properties, Random}
|
||||||
|
@ -47,9 +45,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
props.put("log.retention.check.interval.ms", (5 * 1000 * 60).toString)
|
props.put("log.retention.check.interval.ms", (5 * 1000 * 60).toString)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testGetOffsetsForUnknownTopic(): Unit = {
|
||||||
def testGetOffsetsForUnknownTopic(quorum: String): Unit = {
|
|
||||||
val topicPartition = new TopicPartition("foo", 0)
|
val topicPartition = new TopicPartition("foo", 0)
|
||||||
val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
|
val request = ListOffsetsRequest.Builder.forConsumer(false, IsolationLevel.READ_UNCOMMITTED)
|
||||||
.setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP).asJava).build(1)
|
.setTargetTimes(buildTargetTimes(topicPartition, ListOffsetsRequest.LATEST_TIMESTAMP).asJava).build(1)
|
||||||
|
@ -58,9 +55,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
}
|
}
|
||||||
|
|
||||||
@deprecated("ListOffsetsRequest V0", since = "")
|
@deprecated("ListOffsetsRequest V0", since = "")
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testGetOffsetsAfterDeleteRecords(): Unit = {
|
||||||
def testGetOffsetsAfterDeleteRecords(quorum: String): Unit = {
|
|
||||||
val topic = "kafka-"
|
val topic = "kafka-"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val log = createTopicAndGetLog(topic, topicPartition)
|
val log = createTopicAndGetLog(topic, topicPartition)
|
||||||
|
@ -84,9 +80,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
assertEquals(20L, consumerOffset)
|
assertEquals(20L, consumerOffset)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testFetchOffsetByTimestampForMaxTimestampAfterTruncate(): Unit = {
|
||||||
def testFetchOffsetByTimestampForMaxTimestampAfterTruncate(quorum: String): Unit = {
|
|
||||||
val topic = "kafka-"
|
val topic = "kafka-"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val log = createTopicAndGetLog(topic, topicPartition)
|
val log = createTopicAndGetLog(topic, topicPartition)
|
||||||
|
@ -106,9 +101,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
assertEquals(Optional.empty, log.fetchOffsetByTimestamp(ListOffsetsRequest.MAX_TIMESTAMP, Optional.empty).timestampAndOffsetOpt)
|
assertEquals(Optional.empty, log.fetchOffsetByTimestamp(ListOffsetsRequest.MAX_TIMESTAMP, Optional.empty).timestampAndOffsetOpt)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testFetchOffsetByTimestampForMaxTimestampWithUnorderedTimestamps(): Unit = {
|
||||||
def testFetchOffsetByTimestampForMaxTimestampWithUnorderedTimestamps(quorum: String): Unit = {
|
|
||||||
val topic = "kafka-"
|
val topic = "kafka-"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val log = createTopicAndGetLog(topic, topicPartition)
|
val log = createTopicAndGetLog(topic, topicPartition)
|
||||||
|
@ -125,9 +119,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
assertEquals(6L, maxTimestampOffset.get.timestamp)
|
assertEquals(6L, maxTimestampOffset.get.timestamp)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testGetOffsetsBeforeLatestTime(): Unit = {
|
||||||
def testGetOffsetsBeforeLatestTime(quorum: String): Unit = {
|
|
||||||
val topic = "kafka-"
|
val topic = "kafka-"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val log = createTopicAndGetLog(topic, topicPartition)
|
val log = createTopicAndGetLog(topic, topicPartition)
|
||||||
|
@ -158,9 +151,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
assertFalse(FetchResponse.recordsOrFail(fetchResponse.responseData(topicNames, ApiKeys.FETCH.latestVersion).get(topicPartition)).batches.iterator.hasNext)
|
assertFalse(FetchResponse.recordsOrFail(fetchResponse.responseData(topicNames, ApiKeys.FETCH.latestVersion).get(topicPartition)).batches.iterator.hasNext)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testEmptyLogsGetOffsets(): Unit = {
|
||||||
def testEmptyLogsGetOffsets(quorum: String): Unit = {
|
|
||||||
val random = new Random
|
val random = new Random
|
||||||
val topic = "kafka-"
|
val topic = "kafka-"
|
||||||
val topicPartition = new TopicPartition(topic, random.nextInt(10))
|
val topicPartition = new TopicPartition(topic, random.nextInt(10))
|
||||||
|
@ -182,9 +174,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
assertFalse(offsetChanged)
|
assertFalse(offsetChanged)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testFetchOffsetByTimestampForMaxTimestampWithEmptyLog(): Unit = {
|
||||||
def testFetchOffsetByTimestampForMaxTimestampWithEmptyLog(quorum: String): Unit = {
|
|
||||||
val topic = "kafka-"
|
val topic = "kafka-"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val log = createTopicAndGetLog(topic, topicPartition)
|
val log = createTopicAndGetLog(topic, topicPartition)
|
||||||
|
@ -195,9 +186,8 @@ class LogOffsetTest extends BaseRequestTest {
|
||||||
assertEquals(new OffsetResultHolder(Optional.empty[FileRecords.TimestampAndOffset]()), log.fetchOffsetByTimestamp(ListOffsetsRequest.MAX_TIMESTAMP, Optional.empty))
|
assertEquals(new OffsetResultHolder(Optional.empty[FileRecords.TimestampAndOffset]()), log.fetchOffsetByTimestamp(ListOffsetsRequest.MAX_TIMESTAMP, Optional.empty))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testGetOffsetsBeforeEarliestTime(): Unit = {
|
||||||
def testGetOffsetsBeforeEarliestTime(quorum: String): Unit = {
|
|
||||||
val random = new Random
|
val random = new Random
|
||||||
val topic = "kafka-"
|
val topic = "kafka-"
|
||||||
val topicPartition = new TopicPartition(topic, random.nextInt(3))
|
val topicPartition = new TopicPartition(topic, random.nextInt(3))
|
||||||
|
|
|
@ -27,9 +27,7 @@ import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerialize
|
||||||
import org.apache.kafka.server.config.ReplicationConfigs
|
import org.apache.kafka.server.config.ReplicationConfigs
|
||||||
import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile
|
import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.io.File
|
import java.io.File
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
@ -104,9 +102,8 @@ class LogRecoveryTest extends QuorumTestHarness {
|
||||||
super.tearDown()
|
super.tearDown()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testHWCheckpointNoFailuresSingleLogSegment(): Unit = {
|
||||||
def testHWCheckpointNoFailuresSingleLogSegment(quorum: String): Unit = {
|
|
||||||
val numMessages = 2L
|
val numMessages = 2L
|
||||||
sendMessages(numMessages.toInt)
|
sendMessages(numMessages.toInt)
|
||||||
|
|
||||||
|
@ -122,9 +119,8 @@ class LogRecoveryTest extends QuorumTestHarness {
|
||||||
assertEquals(numMessages, followerHW)
|
assertEquals(numMessages, followerHW)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testHWCheckpointWithFailuresSingleLogSegment(): Unit = {
|
||||||
def testHWCheckpointWithFailuresSingleLogSegment(quorum: String): Unit = {
|
|
||||||
var leader = getLeaderIdForPartition(servers, topicPartition)
|
var leader = getLeaderIdForPartition(servers, topicPartition)
|
||||||
|
|
||||||
assertEquals(0L, hwFile1.read().getOrDefault(topicPartition, 0L))
|
assertEquals(0L, hwFile1.read().getOrDefault(topicPartition, 0L))
|
||||||
|
@ -183,9 +179,8 @@ class LogRecoveryTest extends QuorumTestHarness {
|
||||||
assertEquals(hw, hwFile2.read().getOrDefault(topicPartition, 0L))
|
assertEquals(hw, hwFile2.read().getOrDefault(topicPartition, 0L))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testHWCheckpointNoFailuresMultipleLogSegments(): Unit = {
|
||||||
def testHWCheckpointNoFailuresMultipleLogSegments(quorum: String): Unit = {
|
|
||||||
sendMessages(20)
|
sendMessages(20)
|
||||||
val hw = 20L
|
val hw = 20L
|
||||||
// give some time for follower 1 to record leader HW of 600
|
// give some time for follower 1 to record leader HW of 600
|
||||||
|
@ -200,9 +195,8 @@ class LogRecoveryTest extends QuorumTestHarness {
|
||||||
assertEquals(hw, followerHW)
|
assertEquals(hw, followerHW)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testHWCheckpointWithFailuresMultipleLogSegments(): Unit = {
|
||||||
def testHWCheckpointWithFailuresMultipleLogSegments(quorum: String): Unit = {
|
|
||||||
var leader = getLeaderIdForPartition(servers, topicPartition)
|
var leader = getLeaderIdForPartition(servers, topicPartition)
|
||||||
|
|
||||||
sendMessages(2)
|
sendMessages(2)
|
||||||
|
|
|
@ -27,9 +27,7 @@ import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse}
|
||||||
import org.apache.kafka.metadata.BrokerState
|
import org.apache.kafka.metadata.BrokerState
|
||||||
import org.apache.kafka.test.TestUtils.isValidClusterId
|
import org.apache.kafka.test.TestUtils.isValidClusterId
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.collection.Seq
|
import scala.collection.Seq
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -41,24 +39,21 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
doSetup(testInfo, createOffsetsTopic = false)
|
doSetup(testInfo, createOffsetsTopic = false)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testClusterIdWithRequestVersion1(): Unit = {
|
||||||
def testClusterIdWithRequestVersion1(quorum: String): Unit = {
|
|
||||||
val v1MetadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort))
|
val v1MetadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(1.toShort))
|
||||||
val v1ClusterId = v1MetadataResponse.clusterId
|
val v1ClusterId = v1MetadataResponse.clusterId
|
||||||
assertNull(v1ClusterId, s"v1 clusterId should be null")
|
assertNull(v1ClusterId, s"v1 clusterId should be null")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testClusterIdIsValid(): Unit = {
|
||||||
def testClusterIdIsValid(quorum: String): Unit = {
|
|
||||||
val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(4.toShort))
|
val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(4.toShort))
|
||||||
isValidClusterId(metadataResponse.clusterId)
|
isValidClusterId(metadataResponse.clusterId)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testRack(): Unit = {
|
||||||
def testRack(quorum: String): Unit = {
|
|
||||||
val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(4.toShort))
|
val metadataResponse = sendMetadataRequest(MetadataRequest.Builder.allTopics.build(4.toShort))
|
||||||
// Validate rack matches what's set in generateConfigs() above
|
// Validate rack matches what's set in generateConfigs() above
|
||||||
metadataResponse.brokers.forEach { broker =>
|
metadataResponse.brokers.forEach { broker =>
|
||||||
|
@ -66,9 +61,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIsInternal(): Unit = {
|
||||||
def testIsInternal(quorum: String): Unit = {
|
|
||||||
val internalTopic = Topic.GROUP_METADATA_TOPIC_NAME
|
val internalTopic = Topic.GROUP_METADATA_TOPIC_NAME
|
||||||
val notInternalTopic = "notInternal"
|
val notInternalTopic = "notInternal"
|
||||||
// create the topics
|
// create the topics
|
||||||
|
@ -88,9 +82,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
assertEquals(Set(internalTopic).asJava, metadataResponse.buildCluster().internalTopics)
|
assertEquals(Set(internalTopic).asJava, metadataResponse.buildCluster().internalTopics)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testNoTopicsRequest(): Unit = {
|
||||||
def testNoTopicsRequest(quorum: String): Unit = {
|
|
||||||
// create some topics
|
// create some topics
|
||||||
createTopic("t1", 3, 2)
|
createTopic("t1", 3, 2)
|
||||||
createTopic("t2", 3, 2)
|
createTopic("t2", 3, 2)
|
||||||
|
@ -100,9 +93,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
assertTrue(metadataResponse.topicMetadata.isEmpty, "Response should have no topics")
|
assertTrue(metadataResponse.topicMetadata.isEmpty, "Response should have no topics")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAutoTopicCreation(): Unit = {
|
||||||
def testAutoTopicCreation(quorum: String): Unit = {
|
|
||||||
val topic1 = "t1"
|
val topic1 = "t1"
|
||||||
val topic2 = "t2"
|
val topic2 = "t2"
|
||||||
val topic3 = "t3"
|
val topic3 = "t3"
|
||||||
|
@ -128,9 +120,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response3.errors.get(topic5))
|
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION, response3.errors.get(topic5))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAutoCreateTopicWithInvalidReplicationFactor(): Unit = {
|
||||||
def testAutoCreateTopicWithInvalidReplicationFactor(quorum: String): Unit = {
|
|
||||||
// Shutdown all but one broker so that the number of brokers is less than the default replication factor
|
// Shutdown all but one broker so that the number of brokers is less than the default replication factor
|
||||||
brokers.tail.foreach(_.shutdown())
|
brokers.tail.foreach(_.shutdown())
|
||||||
brokers.tail.foreach(_.awaitShutdown())
|
brokers.tail.foreach(_.awaitShutdown())
|
||||||
|
@ -144,9 +135,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
assertEquals(0, topicMetadata.partitionMetadata.size)
|
assertEquals(0, topicMetadata.partitionMetadata.size)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAllTopicsRequest(): Unit = {
|
||||||
def testAllTopicsRequest(quorum: String): Unit = {
|
|
||||||
// create some topics
|
// create some topics
|
||||||
createTopic("t1", 3, 2)
|
createTopic("t1", 3, 2)
|
||||||
createTopic("t2", 3, 2)
|
createTopic("t2", 3, 2)
|
||||||
|
@ -162,9 +152,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
assertEquals(2, metadataResponseV1.topicMetadata.size(), "V1 Response should have 2 (all) topics")
|
assertEquals(2, metadataResponseV1.topicMetadata.size(), "V1 Response should have 2 (all) topics")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testTopicIdsInResponse(): Unit = {
|
||||||
def testTopicIdsInResponse(quorum: String): Unit = {
|
|
||||||
val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1))
|
val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1))
|
||||||
val topic1 = "topic1"
|
val topic1 = "topic1"
|
||||||
val topic2 = "topic2"
|
val topic2 = "topic2"
|
||||||
|
@ -192,9 +181,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
/**
|
/**
|
||||||
* Preferred replica should be the first item in the replicas list
|
* Preferred replica should be the first item in the replicas list
|
||||||
*/
|
*/
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testPreferredReplica(): Unit = {
|
||||||
def testPreferredReplica(quorum: String): Unit = {
|
|
||||||
val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1))
|
val replicaAssignment = Map(0 -> Seq(1, 2, 0), 1 -> Seq(2, 0, 1))
|
||||||
createTopicWithAssignment("t1", replicaAssignment)
|
createTopicWithAssignment("t1", replicaAssignment)
|
||||||
// Test metadata on two different brokers to ensure that metadata propagation works correctly
|
// Test metadata on two different brokers to ensure that metadata propagation works correctly
|
||||||
|
@ -216,9 +204,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testReplicaDownResponse(): Unit = {
|
||||||
def testReplicaDownResponse(quorum: String): Unit = {
|
|
||||||
val replicaDownTopic = "replicaDown"
|
val replicaDownTopic = "replicaDown"
|
||||||
val replicaCount = 3
|
val replicaCount = 3
|
||||||
|
|
||||||
|
@ -262,9 +249,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
assertEquals(replicaCount, v1PartitionMetadata.replicaIds.size, s"Response should have $replicaCount replicas")
|
assertEquals(replicaCount, v1PartitionMetadata.replicaIds.size, s"Response should have $replicaCount replicas")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testIsrAfterBrokerShutDownAndJoinsBack(): Unit = {
|
||||||
def testIsrAfterBrokerShutDownAndJoinsBack(quorum: String): Unit = {
|
|
||||||
def checkIsr[B <: KafkaBroker](
|
def checkIsr[B <: KafkaBroker](
|
||||||
brokers: Seq[B],
|
brokers: Seq[B],
|
||||||
topic: String
|
topic: String
|
||||||
|
@ -300,9 +286,8 @@ class MetadataRequestTest extends AbstractMetadataRequestTest {
|
||||||
checkIsr(brokers, topic)
|
checkIsr(brokers, topic)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testAliveBrokersWithNoTopics(): Unit = {
|
||||||
def testAliveBrokersWithNoTopics(quorum: String): Unit = {
|
|
||||||
def checkMetadata[B <: KafkaBroker](
|
def checkMetadata[B <: KafkaBroker](
|
||||||
brokers: Seq[B],
|
brokers: Seq[B],
|
||||||
expectedBrokersCount: Int
|
expectedBrokersCount: Int
|
||||||
|
|
|
@ -26,16 +26,14 @@ import org.apache.kafka.common.protocol.Errors
|
||||||
import org.apache.kafka.common.record.RecordBatch
|
import org.apache.kafka.common.record.RecordBatch
|
||||||
import org.apache.kafka.common.requests.{OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse}
|
import org.apache.kafka.common.requests.{OffsetsForLeaderEpochRequest, OffsetsForLeaderEpochResponse}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
|
||||||
class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
|
class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testOffsetsForLeaderEpochErrorCodes(): Unit = {
|
||||||
def testOffsetsForLeaderEpochErrorCodes(quorum: String): Unit = {
|
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val partition = new TopicPartition(topic, 0)
|
val partition = new TopicPartition(topic, 0)
|
||||||
val epochs = offsetForLeaderTopicCollectionFor(partition, 0, RecordBatch.NO_PARTITION_LEADER_EPOCH)
|
val epochs = offsetForLeaderTopicCollectionFor(partition, 0, RecordBatch.NO_PARTITION_LEADER_EPOCH)
|
||||||
|
@ -57,9 +55,8 @@ class OffsetsForLeaderEpochRequestTest extends BaseRequestTest {
|
||||||
assertResponseError(Errors.NOT_LEADER_OR_FOLLOWER, nonReplica, request)
|
assertResponseError(Errors.NOT_LEADER_OR_FOLLOWER, nonReplica, request)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCurrentEpochValidation(): Unit = {
|
||||||
def testCurrentEpochValidation(quorum: String): Unit = {
|
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val topicPartition = new TopicPartition(topic, 0)
|
val topicPartition = new TopicPartition(topic, 0)
|
||||||
val partitionToLeader = createTopic(topic, replicationFactor = 3)
|
val partitionToLeader = createTopic(topic, replicationFactor = 3)
|
||||||
|
|
|
@ -32,9 +32,9 @@ import org.apache.kafka.server.metrics.KafkaYammerMetrics
|
||||||
import org.apache.kafka.server.record.BrokerCompressionType
|
import org.apache.kafka.server.record.BrokerCompressionType
|
||||||
import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics
|
import org.apache.kafka.storage.log.metrics.BrokerTopicMetrics
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
|
import org.junit.jupiter.api.Test
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
|
import org.junit.jupiter.params.provider.{Arguments, MethodSource}
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.util.concurrent.TimeUnit
|
import java.util.concurrent.TimeUnit
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
|
@ -47,9 +47,8 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
|
|
||||||
val metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala
|
val metricsKeySet = KafkaYammerMetrics.defaultRegistry.allMetrics.keySet.asScala
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testSimpleProduceRequest(): Unit = {
|
||||||
def testSimpleProduceRequest(quorum: String): Unit = {
|
|
||||||
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
|
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
|
||||||
|
|
||||||
def sendAndCheck(memoryRecords: MemoryRecords, expectedOffset: Long): Unit = {
|
def sendAndCheck(memoryRecords: MemoryRecords, expectedOffset: Long): Unit = {
|
||||||
|
@ -163,9 +162,8 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
assertEquals("One or more records have been rejected due to invalid timestamp", partitionProduceResponse.errorMessage)
|
assertEquals("One or more records have been rejected due to invalid timestamp", partitionProduceResponse.errorMessage)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testProduceToNonReplica(): Unit = {
|
||||||
def testProduceToNonReplica(quorum: String): Unit = {
|
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val partition = 0
|
val partition = 0
|
||||||
|
|
||||||
|
@ -212,9 +210,8 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
}.getOrElse(throw new AssertionError(s"No leader elected for topic $topic"))
|
}.getOrElse(throw new AssertionError(s"No leader elected for topic $topic"))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCorruptLz4ProduceRequest(): Unit = {
|
||||||
def testCorruptLz4ProduceRequest(quorum: String): Unit = {
|
|
||||||
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
|
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
|
||||||
val topicId = getTopicIds().get("topic").get
|
val topicId = getTopicIds().get("topic").get
|
||||||
val timestamp = 1000000
|
val timestamp = 1000000
|
||||||
|
@ -247,9 +244,8 @@ class ProduceRequestTest extends BaseRequestTest {
|
||||||
assertTrue(TestUtils.meterCount(s"${BrokerTopicMetrics.INVALID_MESSAGE_CRC_RECORDS_PER_SEC}") > 0)
|
assertTrue(TestUtils.meterCount(s"${BrokerTopicMetrics.INVALID_MESSAGE_CRC_RECORDS_PER_SEC}") > 0)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testZSTDProduceRequest(): Unit = {
|
||||||
def testZSTDProduceRequest(quorum: String): Unit = {
|
|
||||||
val topic = "topic"
|
val topic = "topic"
|
||||||
val partition = 0
|
val partition = 0
|
||||||
|
|
||||||
|
|
|
@ -17,15 +17,13 @@
|
||||||
|
|
||||||
package kafka.server
|
package kafka.server
|
||||||
|
|
||||||
import org.junit.jupiter.api.AfterEach
|
import org.junit.jupiter.api.{AfterEach, Test}
|
||||||
import kafka.utils.TestUtils
|
import kafka.utils.TestUtils
|
||||||
import TestUtils._
|
import TestUtils._
|
||||||
import kafka.api.IntegrationTestHarness
|
import kafka.api.IntegrationTestHarness
|
||||||
import org.apache.kafka.clients.producer.ProducerRecord
|
import org.apache.kafka.clients.producer.ProducerRecord
|
||||||
import org.apache.kafka.common.TopicPartition
|
import org.apache.kafka.common.TopicPartition
|
||||||
import org.apache.kafka.common.serialization.StringSerializer
|
import org.apache.kafka.common.serialization.StringSerializer
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
class ReplicaFetchTest extends IntegrationTestHarness {
|
class ReplicaFetchTest extends IntegrationTestHarness {
|
||||||
val topic1 = "foo"
|
val topic1 = "foo"
|
||||||
|
@ -39,9 +37,8 @@ class ReplicaFetchTest extends IntegrationTestHarness {
|
||||||
|
|
||||||
override def brokerCount: Int = 2
|
override def brokerCount: Int = 2
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testReplicaFetcherThread(): Unit = {
|
||||||
def testReplicaFetcherThread(quorum: String): Unit = {
|
|
||||||
val partition = 0
|
val partition = 0
|
||||||
val testMessageList1 = List("test1", "test2", "test3", "test4")
|
val testMessageList1 = List("test1", "test2", "test3", "test4")
|
||||||
val testMessageList2 = List("test5", "test6", "test7", "test8")
|
val testMessageList2 = List("test5", "test6", "test7", "test8")
|
||||||
|
|
|
@ -37,9 +37,7 @@ import org.apache.kafka.server.common.{Feature, MetadataVersion}
|
||||||
import org.apache.kafka.server.config.QuotaConfig
|
import org.apache.kafka.server.config.QuotaConfig
|
||||||
import org.apache.kafka.server.quota.QuotaType
|
import org.apache.kafka.server.quota.QuotaType
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.AfterEach
|
import org.junit.jupiter.api.{AfterEach, Test}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.jdk.CollectionConverters._
|
import scala.jdk.CollectionConverters._
|
||||||
import scala.util.Using
|
import scala.util.Using
|
||||||
|
@ -67,15 +65,13 @@ class ReplicationQuotasTest extends QuorumTestHarness {
|
||||||
super.tearDown()
|
super.tearDown()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def shouldBootstrapTwoBrokersWithLeaderThrottle(): Unit = {
|
||||||
def shouldBootstrapTwoBrokersWithLeaderThrottle(quorum: String): Unit = {
|
|
||||||
shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(true)
|
shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def shouldBootstrapTwoBrokersWithFollowerThrottle(): Unit = {
|
||||||
def shouldBootstrapTwoBrokersWithFollowerThrottle(quorum: String): Unit = {
|
|
||||||
shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(false)
|
shouldMatchQuotaReplicatingThroughAnAsymmetricTopology(false)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -194,9 +190,8 @@ class ReplicationQuotasTest extends QuorumTestHarness {
|
||||||
|
|
||||||
def tp(partition: Int): TopicPartition = new TopicPartition(topic, partition)
|
def tp(partition: Int): TopicPartition = new TopicPartition(topic, partition)
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def shouldThrottleOldSegments(): Unit = {
|
||||||
def shouldThrottleOldSegments(quorum: String): Unit = {
|
|
||||||
/**
|
/**
|
||||||
* Simple test which ensures throttled replication works when the dataset spans many segments
|
* Simple test which ensures throttled replication works when the dataset spans many segments
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -43,9 +43,7 @@ import org.apache.kafka.server.authorizer.{Action, AuthorizableRequestContext, A
|
||||||
import org.apache.kafka.server.config.{QuotaConfig, ServerConfigs}
|
import org.apache.kafka.server.config.{QuotaConfig, ServerConfigs}
|
||||||
import org.apache.kafka.server.quota.QuotaType
|
import org.apache.kafka.server.quota.QuotaType
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, TestInfo}
|
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test, TestInfo}
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import java.net.InetAddress
|
import java.net.InetAddress
|
||||||
import java.util
|
import java.util
|
||||||
|
@ -133,32 +131,28 @@ class RequestQuotaTest extends BaseRequestTest {
|
||||||
finally super.tearDown()
|
finally super.tearDown()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testResponseThrottleTime(): Unit = {
|
||||||
def testResponseThrottleTime(quorum: String): Unit = {
|
|
||||||
for (apiKey <- clientActions ++ clusterActionsWithThrottleForBroker)
|
for (apiKey <- clientActions ++ clusterActionsWithThrottleForBroker)
|
||||||
submitTest(apiKey, () => checkRequestThrottleTime(apiKey))
|
submitTest(apiKey, () => checkRequestThrottleTime(apiKey))
|
||||||
|
|
||||||
waitAndCheckResults()
|
waitAndCheckResults()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testResponseThrottleTimeWhenBothProduceAndRequestQuotasViolated(): Unit = {
|
||||||
def testResponseThrottleTimeWhenBothProduceAndRequestQuotasViolated(quorum: String): Unit = {
|
|
||||||
submitTest(ApiKeys.PRODUCE, () => checkSmallQuotaProducerRequestThrottleTime())
|
submitTest(ApiKeys.PRODUCE, () => checkSmallQuotaProducerRequestThrottleTime())
|
||||||
waitAndCheckResults()
|
waitAndCheckResults()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testResponseThrottleTimeWhenBothFetchAndRequestQuotasViolated(): Unit = {
|
||||||
def testResponseThrottleTimeWhenBothFetchAndRequestQuotasViolated(quorum: String): Unit = {
|
|
||||||
submitTest(ApiKeys.FETCH, () => checkSmallQuotaConsumerRequestThrottleTime())
|
submitTest(ApiKeys.FETCH, () => checkSmallQuotaConsumerRequestThrottleTime())
|
||||||
waitAndCheckResults()
|
waitAndCheckResults()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUnthrottledClient(): Unit = {
|
||||||
def testUnthrottledClient(quorum: String): Unit = {
|
|
||||||
for (apiKey <- clientActions) {
|
for (apiKey <- clientActions) {
|
||||||
submitTest(apiKey, () => checkUnthrottledClient(apiKey))
|
submitTest(apiKey, () => checkUnthrottledClient(apiKey))
|
||||||
}
|
}
|
||||||
|
@ -166,9 +160,8 @@ class RequestQuotaTest extends BaseRequestTest {
|
||||||
waitAndCheckResults()
|
waitAndCheckResults()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testExemptRequestTime(): Unit = {
|
||||||
def testExemptRequestTime(quorum: String): Unit = {
|
|
||||||
// Exclude `DESCRIBE_QUORUM`, maybe it shouldn't be a cluster action
|
// Exclude `DESCRIBE_QUORUM`, maybe it shouldn't be a cluster action
|
||||||
val actions = clusterActions -- clusterActionsWithThrottleForBroker -- RequestQuotaTest.Envelope -- RequestQuotaTest.ShareGroupState - ApiKeys.DESCRIBE_QUORUM
|
val actions = clusterActions -- clusterActionsWithThrottleForBroker -- RequestQuotaTest.Envelope -- RequestQuotaTest.ShareGroupState - ApiKeys.DESCRIBE_QUORUM
|
||||||
for (apiKey <- actions) {
|
for (apiKey <- actions) {
|
||||||
|
@ -178,9 +171,8 @@ class RequestQuotaTest extends BaseRequestTest {
|
||||||
waitAndCheckResults()
|
waitAndCheckResults()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testUnauthorizedThrottle(): Unit = {
|
||||||
def testUnauthorizedThrottle(quorum: String): Unit = {
|
|
||||||
RequestQuotaTest.principal = RequestQuotaTest.UnauthorizedPrincipal
|
RequestQuotaTest.principal = RequestQuotaTest.UnauthorizedPrincipal
|
||||||
|
|
||||||
val apiKeys = ApiKeys.brokerApis
|
val apiKeys = ApiKeys.brokerApis
|
||||||
|
|
|
@ -29,11 +29,11 @@ import org.apache.kafka.common.utils.Exit
|
||||||
import org.apache.kafka.metadata.BrokerState
|
import org.apache.kafka.metadata.BrokerState
|
||||||
import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
|
import org.apache.kafka.server.config.{KRaftConfigs, ServerLogConfigs}
|
||||||
import org.apache.kafka.storage.internals.log.LogManager
|
import org.apache.kafka.storage.internals.log.LogManager
|
||||||
import org.junit.jupiter.api.{BeforeEach, TestInfo, Timeout}
|
import org.junit.jupiter.api.{BeforeEach, Test, TestInfo, Timeout}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.api.function.Executable
|
import org.junit.jupiter.api.function.Executable
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
import org.junit.jupiter.params.ParameterizedTest
|
||||||
import org.junit.jupiter.params.provider.{MethodSource, ValueSource}
|
import org.junit.jupiter.params.provider.MethodSource
|
||||||
|
|
||||||
import java.time.Duration
|
import java.time.Duration
|
||||||
import java.util.Properties
|
import java.util.Properties
|
||||||
|
@ -134,18 +134,16 @@ class ServerShutdownTest extends KafkaServerTestHarness {
|
||||||
producer.close()
|
producer.close()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testCleanShutdownAfterFailedStartup(): Unit = {
|
||||||
def testCleanShutdownAfterFailedStartup(quorum: String): Unit = {
|
|
||||||
propsToChangeUponRestart.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "1000")
|
propsToChangeUponRestart.setProperty(KRaftConfigs.INITIAL_BROKER_REGISTRATION_TIMEOUT_MS_CONFIG, "1000")
|
||||||
shutdownBroker()
|
shutdownBroker()
|
||||||
shutdownKRaftController()
|
shutdownKRaftController()
|
||||||
verifyCleanShutdownAfterFailedStartup[CancellationException]
|
verifyCleanShutdownAfterFailedStartup[CancellationException]
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testNoCleanShutdownAfterFailedStartupDueToCorruptLogs(): Unit = {
|
||||||
def testNoCleanShutdownAfterFailedStartupDueToCorruptLogs(quorum: String): Unit = {
|
|
||||||
createTopic(topic)
|
createTopic(topic)
|
||||||
shutdownBroker()
|
shutdownBroker()
|
||||||
config.logDirs.foreach { dirName =>
|
config.logDirs.foreach { dirName =>
|
||||||
|
@ -174,9 +172,8 @@ class ServerShutdownTest extends KafkaServerTestHarness {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testShutdownWithKRaftControllerUnavailable(): Unit = {
|
||||||
def testShutdownWithKRaftControllerUnavailable(quorum: String): Unit = {
|
|
||||||
shutdownKRaftController()
|
shutdownKRaftController()
|
||||||
killBroker(0, Duration.ofSeconds(1))
|
killBroker(0, Duration.ofSeconds(1))
|
||||||
CoreUtils.delete(broker.config.logDirs)
|
CoreUtils.delete(broker.config.logDirs)
|
||||||
|
@ -220,9 +217,8 @@ class ServerShutdownTest extends KafkaServerTestHarness {
|
||||||
.count(isNonDaemonKafkaThread))
|
.count(isNonDaemonKafkaThread))
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def testConsecutiveShutdown(): Unit = {
|
||||||
def testConsecutiveShutdown(quorum: String): Unit = {
|
|
||||||
shutdownBroker()
|
shutdownBroker()
|
||||||
brokers.head.shutdown()
|
brokers.head.shutdown()
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,10 +34,8 @@ import org.apache.kafka.common.serialization.StringSerializer
|
||||||
import org.apache.kafka.common.utils.{LogContext, Time}
|
import org.apache.kafka.common.utils.{LogContext, Time}
|
||||||
import org.apache.kafka.server.network.BrokerEndPoint
|
import org.apache.kafka.server.network.BrokerEndPoint
|
||||||
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
import org.apache.kafka.test.{TestUtils => JTestUtils}
|
||||||
import org.junit.jupiter.api.AfterEach
|
import org.junit.jupiter.api.{AfterEach, Test}
|
||||||
import org.junit.jupiter.api.Assertions._
|
import org.junit.jupiter.api.Assertions._
|
||||||
import org.junit.jupiter.params.ParameterizedTest
|
|
||||||
import org.junit.jupiter.params.provider.ValueSource
|
|
||||||
|
|
||||||
import scala.collection.mutable.ListBuffer
|
import scala.collection.mutable.ListBuffer
|
||||||
import scala.collection.{Map, Seq}
|
import scala.collection.{Map, Seq}
|
||||||
|
@ -64,9 +62,8 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
|
||||||
super.tearDown()
|
super.tearDown()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def shouldAddCurrentLeaderEpochToMessagesAsTheyAreWrittenToLeader(): Unit = {
|
||||||
def shouldAddCurrentLeaderEpochToMessagesAsTheyAreWrittenToLeader(quorum: String): Unit = {
|
|
||||||
brokers ++= (0 to 1).map { id => createBroker(fromProps(createBrokerConfig(id))) }
|
brokers ++= (0 to 1).map { id => createBroker(fromProps(createBrokerConfig(id))) }
|
||||||
|
|
||||||
// Given two topics with replication of a single partition
|
// Given two topics with replication of a single partition
|
||||||
|
@ -97,9 +94,8 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
|
||||||
waitUntilTrue(() => messagesHaveLeaderEpoch(brokers(0), expectedLeaderEpoch, 4), "Leader epoch should be 1")
|
waitUntilTrue(() => messagesHaveLeaderEpoch(brokers(0), expectedLeaderEpoch, 4), "Leader epoch should be 1")
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def shouldSendLeaderEpochRequestAndGetAResponse(): Unit = {
|
||||||
def shouldSendLeaderEpochRequestAndGetAResponse(quorum: String): Unit = {
|
|
||||||
|
|
||||||
//3 brokers, put partition on 100/101 and then pretend to be 102
|
//3 brokers, put partition on 100/101 and then pretend to be 102
|
||||||
brokers ++= (100 to 102).map { id => createBroker(fromProps(createBrokerConfig(id))) }
|
brokers ++= (100 to 102).map { id => createBroker(fromProps(createBrokerConfig(id))) }
|
||||||
|
@ -145,9 +141,8 @@ class LeaderEpochIntegrationTest extends QuorumTestHarness with Logging {
|
||||||
fetcher1.close()
|
fetcher1.close()
|
||||||
}
|
}
|
||||||
|
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = Array("kraft"))
|
def shouldIncreaseLeaderEpochBetweenLeaderRestarts(): Unit = {
|
||||||
def shouldIncreaseLeaderEpochBetweenLeaderRestarts(quorum: String): Unit = {
|
|
||||||
//Setup: we are only interested in the single partition on broker 101
|
//Setup: we are only interested in the single partition on broker 101
|
||||||
brokers += createBroker(fromProps(createBrokerConfig(100)))
|
brokers += createBroker(fromProps(createBrokerConfig(100)))
|
||||||
assertEquals(controllerServer.config.nodeId, waitUntilQuorumLeaderElected(controllerServer))
|
assertEquals(controllerServer.config.nodeId, waitUntilQuorumLeaderElected(controllerServer))
|
||||||
|
|
|
@ -21,8 +21,7 @@ import kafka.api.AbstractAuthorizerIntegrationTest;
|
||||||
import org.apache.kafka.common.acl.AccessControlEntry;
|
import org.apache.kafka.common.acl.AccessControlEntry;
|
||||||
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
import org.apache.kafka.common.errors.GroupIdNotFoundException;
|
||||||
|
|
||||||
import org.junit.jupiter.params.ParameterizedTest;
|
import org.junit.jupiter.api.Test;
|
||||||
import org.junit.jupiter.params.provider.ValueSource;
|
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
|
@ -35,9 +34,8 @@ import static org.junit.jupiter.api.Assertions.assertInstanceOf;
|
||||||
import static org.junit.jupiter.api.Assertions.fail;
|
import static org.junit.jupiter.api.Assertions.fail;
|
||||||
|
|
||||||
public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
public class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
|
||||||
@ParameterizedTest
|
@Test
|
||||||
@ValueSource(strings = {"kraft"})
|
public void testDescribeGroupCliWithGroupDescribe() throws Exception {
|
||||||
public void testDescribeGroupCliWithGroupDescribe(String quorum) throws Exception {
|
|
||||||
addAndVerifyAcls(CollectionConverters.asScala(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), "*", DESCRIBE, ALLOW))).toSet(), groupResource());
|
addAndVerifyAcls(CollectionConverters.asScala(Collections.singleton(new AccessControlEntry(ClientPrincipal().toString(), "*", DESCRIBE, ALLOW))).toSet(), groupResource());
|
||||||
|
|
||||||
String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};
|
String[] cgcArgs = new String[]{"--bootstrap-server", bootstrapServers(listenerName()), "--describe", "--group", group()};
|
||||||
|
|
Loading…
Reference in New Issue