diff --git a/.github/scripts/junit.py b/.github/scripts/junit.py index b23f444720ba3..5a2088f8ea196 100644 --- a/.github/scripts/junit.py +++ b/.github/scripts/junit.py @@ -91,7 +91,10 @@ class TestSuite: def clean_test_name(test_name: str) -> str: cleaned = test_name.strip("\"").rstrip("()") m = method_matcher.match(cleaned) - return m.group(1) + if m is None: + raise ValueError(f"Could not parse test name '{test_name}'. Expected a valid Java method name.") + else: + return m.group(1) class TestCatalogExporter: diff --git a/build.gradle b/build.gradle index ecc786b2d5f77..90e79f7fe73b1 100644 --- a/build.gradle +++ b/build.gradle @@ -49,7 +49,7 @@ ext { gradleVersion = versions.gradle minClientJavaVersion = 11 minNonClientJavaVersion = 17 - modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams-scala", ":test-common:test-common-runtime"] + modulesNeedingJava11 = [":clients", ":generator", ":streams", ":streams:test-utils", ":streams-scala", ":test-common:test-common-util"] buildVersionFileName = "kafka-version.properties" @@ -139,10 +139,11 @@ ext { runtimeTestLibs = [ libs.slf4jLog4j2, libs.junitPlatformLanucher, - project(":test-common:test-common-runtime") + libs.jacksonDatabindYaml, + project(":test-common:test-common-util") ] - log4jRuntimeLibs = [ + log4jReleaseLibs = [ libs.slf4jLog4j2, libs.log4j1Bridge2Api, libs.jacksonDatabindYaml @@ -1059,7 +1060,7 @@ project(':core') { } dependencies { - releaseOnly log4jRuntimeLibs + releaseOnly log4jReleaseLibs // `core` is often used in users' tests, define the following dependencies as `api` for backwards compatibility // even though the `core` module doesn't expose any public API api project(':clients') @@ -1102,8 +1103,9 @@ project(':core') { testImplementation project(':server-common').sourceSets.test.output testImplementation project(':storage:storage-api').sourceSets.test.output testImplementation project(':server').sourceSets.test.output - testImplementation project(':test-common') - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-runtime') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-util') testImplementation libs.bcpkix testImplementation libs.mockitoCore testImplementation(libs.apacheda) { @@ -1534,21 +1536,17 @@ project(':group-coordinator') { srcJar.dependsOn 'processMessages' } -project(':test-common') { - // Test framework stuff. Implementations that support test-common-api + +project(':test-common:test-common-internal-api') { + // Interfaces, config classes, and other test APIs. Java 17 only base { - archivesName = "kafka-test-common" + archivesName = "kafka-test-common-internal-api" } dependencies { - implementation project(':core') - implementation project(':metadata') - implementation project(':server') - implementation project(':raft') - implementation project(':storage') - implementation project(':server-common') - implementation libs.jacksonDatabindYaml - implementation libs.slf4jApi + implementation project(':server-common') // Only project dependency allowed + + implementation libs.junitJupiterApi testImplementation libs.junitJupiter testImplementation libs.mockitoCore @@ -1558,7 +1556,7 @@ project(':test-common') { } checkstyle { - configProperties = checkstyleConfigProperties("import-control-test-common.xml") + configProperties = checkstyleConfigProperties("import-control-test-common-internal-api.xml") } javadoc { @@ -1566,33 +1564,22 @@ project(':test-common') { } } -project(':test-common:test-common-api') { - // Interfaces, config classes, and other test APIs +project(':test-common:test-common-util') { + // Runtime-only JUnit extensions for entire project. Java 11 only base { - archivesName = "kafka-test-common-api" + archivesName = "kafka-test-common-util" } dependencies { - implementation project(':clients') - implementation project(':core') - implementation project(':group-coordinator') - implementation project(':metadata') - implementation project(':raft') - implementation project(':server') - implementation project(':server-common') - implementation project(':storage') - implementation project(':test-common') + implementation libs.junitPlatformLanucher implementation libs.junitJupiterApi - - testImplementation libs.junitJupiter - testImplementation libs.mockitoCore + implementation libs.junitJupiter + implementation libs.slf4jApi testImplementation testLog4j2Libs - - testRuntimeOnly runtimeTestLibs } checkstyle { - configProperties = checkstyleConfigProperties("import-control-test-common-api.xml") + configProperties = checkstyleConfigProperties("import-control-test-common-util.xml") } javadoc { @@ -1601,21 +1588,36 @@ project(':test-common:test-common-api') { } project(':test-common:test-common-runtime') { - // Runtime-only test code including JUnit extentions + // Runtime-only JUnit extensions for integration tests. Java 17 only base { archivesName = "kafka-test-common-runtime" } dependencies { + implementation project(':test-common:test-common-internal-api') + implementation project(':clients') + implementation project(':core') + implementation project(':group-coordinator') + implementation project(':metadata') + implementation project(':raft') + implementation project(':server') + implementation project(':server-common') + implementation project(':storage') + implementation libs.junitPlatformLanucher - implementation libs.junitJupiterApi implementation libs.junitJupiter + implementation libs.jacksonDatabindYaml implementation libs.slf4jApi + + testImplementation libs.junitJupiter + testImplementation libs.mockitoCore testImplementation testLog4j2Libs + + testRuntimeOnly runtimeTestLibs } checkstyle { - configProperties = checkstyleConfigProperties("import-control-test-common-api.xml") + configProperties = checkstyleConfigProperties("import-control-test-common-runtime.xml") } javadoc { @@ -1642,8 +1644,8 @@ project(':transaction-coordinator') { testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation project(':clients').sourceSets.test.output - testImplementation project(':test-common') - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-runtime') + testImplementation project(':test-common:test-common-internal-api') testRuntimeOnly runtimeTestLibs @@ -1862,6 +1864,7 @@ project(':clients') { compileOnly libs.jose4j // for SASL/OAUTHBEARER JWT validation; only used by broker + testImplementation project(':test-common:test-common-util') testImplementation libs.bcpkix testImplementation libs.jacksonJakartarsJsonProvider testImplementation libs.jose4j @@ -1876,7 +1879,6 @@ project(':clients') { testRuntimeOnly libs.jacksonDatabind testRuntimeOnly libs.jacksonJDK8Datatypes testRuntimeOnly runtimeTestLibs - testRuntimeOnly log4jRuntimeLibs generator project(':generator') } @@ -2264,7 +2266,8 @@ project(':storage') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':core') testImplementation project(':core').sourceSets.test.output - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server') testImplementation project(':server-common') testImplementation project(':server-common').sourceSets.test.output @@ -2421,7 +2424,7 @@ project(':tools') { } dependencies { - releaseOnly log4jRuntimeLibs + releaseOnly log4jReleaseLibs implementation project(':clients') implementation project(':metadata') @@ -2453,7 +2456,8 @@ project(':tools') { testImplementation project(':server').sourceSets.test.output testImplementation project(':core') testImplementation project(':core').sourceSets.test.output - testImplementation project(':test-common:test-common-api') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server-common') testImplementation project(':server-common').sourceSets.test.output testImplementation project(':connect:api') @@ -2464,7 +2468,6 @@ project(':tools') { testImplementation project(':streams') testImplementation project(':streams').sourceSets.test.output testImplementation project(':streams:integration-tests').sourceSets.test.output - testImplementation project(':test-common') testImplementation libs.junitJupiter testImplementation libs.mockitoCore testImplementation libs.mockitoJunitJupiter // supports MockitoExtension @@ -2645,7 +2648,6 @@ project(':streams') { testRuntimeOnly project(':streams:test-utils') testRuntimeOnly runtimeTestLibs - testRuntimeOnly log4jRuntimeLibs generator project(':generator') } @@ -2835,7 +2837,7 @@ project(':streams:integration-tests') { testImplementation project(':storage') testImplementation project(':streams').sourceSets.test.output testImplementation project(':streams:streams-scala') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':tools') testImplementation project(':transaction-coordinator') testImplementation libs.bcpkix @@ -3511,14 +3513,15 @@ project(':connect:runtime') { testImplementation project(':server') testImplementation project(':metadata') testImplementation project(':server-common') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-internal-api') + testImplementation project(':test-common:test-common-util') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server-common') testImplementation project(':server') testImplementation project(':group-coordinator') testImplementation project(':storage') testImplementation project(':connect:test-plugins') testImplementation project(':server-common').sourceSets.test.output - testImplementation project(':test-common:test-common-api') testImplementation libs.jacksonDatabindYaml testImplementation libs.junitJupiter @@ -3632,7 +3635,7 @@ project(':connect:file') { testImplementation project(':connect:runtime') testImplementation project(':connect:runtime').sourceSets.test.output testImplementation project(':core') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server-common').sourceSets.test.output testRuntimeOnly runtimeTestLibs @@ -3736,7 +3739,7 @@ project(':connect:mirror') { testImplementation project(':clients').sourceSets.test.output testImplementation project(':connect:runtime').sourceSets.test.output testImplementation project(':core') - testImplementation project(':test-common') + testImplementation project(':test-common:test-common-runtime') testImplementation project(':server') testImplementation project(':server-common').sourceSets.test.output diff --git a/checkstyle/import-control-coordinator-common.xml b/checkstyle/import-control-coordinator-common.xml index eb107a4b17a51..c08955fd4228a 100644 --- a/checkstyle/import-control-coordinator-common.xml +++ b/checkstyle/import-control-coordinator-common.xml @@ -37,6 +37,7 @@ + diff --git a/checkstyle/import-control-core.xml b/checkstyle/import-control-core.xml index 3cfd0ce663cc1..ce3ae6ea7c544 100644 --- a/checkstyle/import-control-core.xml +++ b/checkstyle/import-control-core.xml @@ -36,6 +36,7 @@ + diff --git a/checkstyle/import-control-group-coordinator.xml b/checkstyle/import-control-group-coordinator.xml index 0619ea444d5f6..19d9d5616135d 100644 --- a/checkstyle/import-control-group-coordinator.xml +++ b/checkstyle/import-control-group-coordinator.xml @@ -45,6 +45,7 @@ + diff --git a/checkstyle/import-control-metadata.xml b/checkstyle/import-control-metadata.xml index d45fbc7d16014..484d6866db682 100644 --- a/checkstyle/import-control-metadata.xml +++ b/checkstyle/import-control-metadata.xml @@ -44,6 +44,7 @@ + diff --git a/checkstyle/import-control-server-common.xml b/checkstyle/import-control-server-common.xml index ac8706686617c..7d7ddb70d87e4 100644 --- a/checkstyle/import-control-server-common.xml +++ b/checkstyle/import-control-server-common.xml @@ -44,6 +44,7 @@ + diff --git a/checkstyle/import-control-server.xml b/checkstyle/import-control-server.xml index f046ceb74a64f..6c3332b1cfd4f 100644 --- a/checkstyle/import-control-server.xml +++ b/checkstyle/import-control-server.xml @@ -47,6 +47,7 @@ + diff --git a/checkstyle/import-control-share-coordinator.xml b/checkstyle/import-control-share-coordinator.xml index d574588ac65c1..aaea93d32e648 100644 --- a/checkstyle/import-control-share-coordinator.xml +++ b/checkstyle/import-control-share-coordinator.xml @@ -40,6 +40,7 @@ + diff --git a/checkstyle/import-control-storage.xml b/checkstyle/import-control-storage.xml index fecc55f98ea11..d0b6524d9e128 100644 --- a/checkstyle/import-control-storage.xml +++ b/checkstyle/import-control-storage.xml @@ -45,6 +45,7 @@ + diff --git a/checkstyle/import-control-test-common-internal-api.xml b/checkstyle/import-control-test-common-internal-api.xml new file mode 100644 index 0000000000000..c86fc3ab831dc --- /dev/null +++ b/checkstyle/import-control-test-common-internal-api.xml @@ -0,0 +1,43 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/checkstyle/import-control-test-common.xml b/checkstyle/import-control-test-common-runtime.xml similarity index 86% rename from checkstyle/import-control-test-common.xml rename to checkstyle/import-control-test-common-runtime.xml index 9520c0b21b86e..f85c117edea32 100644 --- a/checkstyle/import-control-test-common.xml +++ b/checkstyle/import-control-test-common-runtime.xml @@ -19,10 +19,14 @@ --> + + + + + + - - diff --git a/checkstyle/import-control-test-common-api.xml b/checkstyle/import-control-test-common-util.xml similarity index 90% rename from checkstyle/import-control-test-common-api.xml rename to checkstyle/import-control-test-common-util.xml index 7331cc2d35e98..05e0cc0662e3d 100644 --- a/checkstyle/import-control-test-common-api.xml +++ b/checkstyle/import-control-test-common-util.xml @@ -19,8 +19,8 @@ --> + - - - + + diff --git a/checkstyle/import-control-transaction-coordinator.xml b/checkstyle/import-control-transaction-coordinator.xml index 4a61407cfa988..a6a4025079ce5 100644 --- a/checkstyle/import-control-transaction-coordinator.xml +++ b/checkstyle/import-control-transaction-coordinator.xml @@ -36,6 +36,7 @@ + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 9007251d51e50..dc674ab997a6d 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -46,6 +46,9 @@ + + + diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java index 65ff612761e94..1eb9a77a9f1c3 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinatorTest.java @@ -51,6 +51,7 @@ import org.apache.kafka.common.requests.RequestTestUtils; import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; +import org.apache.kafka.common.test.api.Flaky; import org.apache.kafka.common.utils.LogContext; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; @@ -1435,6 +1436,7 @@ public void testWakeupAfterJoinGroupReceivedExternalCompletion() throws Exceptio awaitFirstHeartbeat(heartbeatReceived); } + @Flaky("KAFKA-18310") @Test public void testWakeupAfterSyncGroupSentExternalCompletion() throws Exception { setupCoordinator(); @@ -1471,6 +1473,7 @@ public boolean matches(AbstractRequest body) { awaitFirstHeartbeat(heartbeatReceived); } + @Flaky("KAFKA-18310") @Test public void testWakeupAfterSyncGroupReceived() throws Exception { setupCoordinator(); diff --git a/core/src/test/java/kafka/admin/AdminFenceProducersTest.java b/core/src/test/java/kafka/admin/AdminFenceProducersTest.java index 1017fb73d2c93..eab7cd18e75af 100644 --- a/core/src/test/java/kafka/admin/AdminFenceProducersTest.java +++ b/core/src/test/java/kafka/admin/AdminFenceProducersTest.java @@ -28,17 +28,14 @@ import org.apache.kafka.common.errors.ProducerFencedException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.coordinator.transaction.TransactionLogConfig; import org.apache.kafka.coordinator.transaction.TransactionStateManagerConfig; import org.apache.kafka.server.config.ServerLogConfigs; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -56,7 +53,6 @@ @ClusterConfigProperty(key = TransactionLogConfig.TRANSACTIONS_TOPIC_MIN_ISR_CONFIG, value = "1"), @ClusterConfigProperty(key = TransactionStateManagerConfig.TRANSACTIONS_ABORT_TIMED_OUT_TRANSACTION_CLEANUP_INTERVAL_MS_CONFIG, value = "2000") }) -@ExtendWith(ClusterTestExtensions.class) public class AdminFenceProducersTest { private static final String TOPIC_NAME = "mytopic"; private static final String TXN_ID = "mytxnid"; diff --git a/core/src/test/java/kafka/admin/ClientTelemetryTest.java b/core/src/test/java/kafka/admin/ClientTelemetryTest.java index f701d3f38450e..e43f52f7271a7 100644 --- a/core/src/test/java/kafka/admin/ClientTelemetryTest.java +++ b/core/src/test/java/kafka/admin/ClientTelemetryTest.java @@ -37,16 +37,13 @@ import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.telemetry.ClientTelemetry; import org.apache.kafka.server.telemetry.ClientTelemetryReceiver; -import org.junit.jupiter.api.extension.ExtendWith; - import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -68,7 +65,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class ClientTelemetryTest { @ClusterTest( diff --git a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java index 5731692f98e1d..31d0de824f176 100644 --- a/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java +++ b/core/src/test/java/kafka/admin/ConfigCommandIntegrationTest.java @@ -25,16 +25,14 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.errors.InvalidConfigurationException; import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mockito; import java.io.ByteArrayOutputStream; @@ -71,7 +69,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; -@ExtendWith(value = ClusterTestExtensions.class) public class ConfigCommandIntegrationTest { private final String defaultBrokerId = "0"; private final String defaultGroupName = "group"; diff --git a/core/src/test/java/kafka/admin/DeleteTopicTest.java b/core/src/test/java/kafka/admin/DeleteTopicTest.java index be87e086f7f5a..522462f4b3be5 100644 --- a/core/src/test/java/kafka/admin/DeleteTopicTest.java +++ b/core/src/test/java/kafka/admin/DeleteTopicTest.java @@ -30,12 +30,11 @@ import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.SimpleRecord; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.TestUtils; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.metadata.BrokerState; @@ -44,8 +43,6 @@ import org.apache.kafka.storage.internals.log.AppendOrigin; import org.apache.kafka.storage.internals.log.VerificationGuard; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -64,7 +61,6 @@ import static org.junit.jupiter.api.Assertions.fail; -@ExtendWith(value = ClusterTestExtensions.class) @ClusterTestDefaults(types = {Type.KRAFT}, brokers = 3, serverProperties = { diff --git a/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java b/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java index 697dda07363a7..8ca269f4b6866 100644 --- a/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java +++ b/core/src/test/java/kafka/admin/UserScramCredentialsCommandTest.java @@ -16,15 +16,12 @@ */ package kafka.admin; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.test.NoRetryException; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -37,7 +34,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @SuppressWarnings("dontUseSystemExit") -@ExtendWith(value = ClusterTestExtensions.class) public class UserScramCredentialsCommandTest { private static final String USER1 = "user1"; private static final String USER2 = "user2"; diff --git a/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java b/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java index 00ba1fbfcad5b..4007f55ee5a44 100644 --- a/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java +++ b/core/src/test/java/kafka/clients/consumer/ConsumerIntegrationTest.java @@ -28,15 +28,12 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.TestUtils; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.ClusterTests; -import org.junit.jupiter.api.extension.ExtendWith; - import java.time.Duration; import java.util.Collection; import java.util.Collections; @@ -45,7 +42,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; -@ExtendWith(ClusterTestExtensions.class) public class ConsumerIntegrationTest { @ClusterTests({ diff --git a/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java b/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java index 75cd070b93d30..dcee16b8ec62c 100644 --- a/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java +++ b/core/src/test/java/kafka/server/BootstrapControllersIntegrationTest.java @@ -51,18 +51,16 @@ import org.apache.kafka.common.resource.PatternType; import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.metadata.authorizer.StandardAuthorizer; import org.apache.kafka.server.common.MetadataVersion; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Timeout; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collection; @@ -88,7 +86,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; @Timeout(120) -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(types = {Type.KRAFT}) public class BootstrapControllersIntegrationTest { private Map adminConfig(ClusterInstance clusterInstance, boolean usingBootstrapControllers) { diff --git a/core/src/test/java/kafka/server/LogManagerIntegrationTest.java b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java index 05b5e8423bab4..3d386283943f5 100644 --- a/core/src/test/java/kafka/server/LogManagerIntegrationTest.java +++ b/core/src/test/java/kafka/server/LogManagerIntegrationTest.java @@ -31,15 +31,12 @@ import org.apache.kafka.common.TopicPartitionInfo; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -55,7 +52,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class LogManagerIntegrationTest { private final ClusterInstance cluster; diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala index 84214a79ed91f..cbd69baedc75c 100644 --- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala @@ -13,9 +13,7 @@ package kafka.api import kafka.log.UnifiedLog -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{Admin, ConsumerGroupDescription} import org.apache.kafka.clients.consumer.{Consumer, GroupProtocol, OffsetAndMetadata} @@ -26,17 +24,16 @@ import org.junit.jupiter.api.Assertions._ import scala.jdk.CollectionConverters._ import org.apache.kafka.common.internals.Topic import org.apache.kafka.common.record.CompressionType +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Timeout -import org.junit.jupiter.api.extension.ExtendWith import java.time.Duration import java.util.Collections import java.util.concurrent.TimeUnit @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class GroupCoordinatorIntegrationTest(cluster: ClusterInstance) { @ClusterTest( diff --git a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala index 5fe06748631cf..e1fa03403b3e4 100644 --- a/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/coordinator/transaction/ProducerIntegrationTest.scala @@ -25,18 +25,17 @@ import org.apache.kafka.clients.producer.{Producer, ProducerConfig, ProducerReco import org.apache.kafka.common.config.TopicConfig import org.apache.kafka.common.errors.RecordTooLargeException import org.apache.kafka.common.TopicPartition -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, ClusterTests, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, ClusterTests, Type} import org.apache.kafka.common.message.InitProducerIdRequestData import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.record.RecordBatch import org.apache.kafka.common.requests.{InitProducerIdRequest, InitProducerIdResponse} -import org.apache.kafka.common.test.TestUtils +import org.apache.kafka.common.test.{ClusterInstance, TestUtils} import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.apache.kafka.server.common.{Feature, MetadataVersion} import org.junit.jupiter.api.Assertions.{assertEquals, assertInstanceOf, assertThrows, assertTrue} -import org.junit.jupiter.api.extension.ExtendWith import java.time.Duration import java.util @@ -53,7 +52,6 @@ import scala.jdk.CollectionConverters._ new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), )) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ProducerIntegrationTest { @ClusterTests(Array( diff --git a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala index fe5a4047d6374..b21badd942252 100644 --- a/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala +++ b/core/src/test/scala/integration/kafka/server/KRaftClusterTest.scala @@ -20,7 +20,6 @@ package kafka.server import kafka.log.UnifiedLog import kafka.network.SocketServer import kafka.server.IntegrationTestUtils.connectAndReceive -import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin._ @@ -37,6 +36,7 @@ import org.apache.kafka.common.quota.ClientQuotaAlteration.Op import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.common.requests.{ApiError, DescribeClusterRequest, DescribeClusterResponse} import org.apache.kafka.common.security.auth.KafkaPrincipal +import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.{Cluster, Endpoint, Reconfigurable, TopicPartition, TopicPartitionInfo} import org.apache.kafka.controller.{QuorumController, QuorumControllerIntegrationTestUtils} import org.apache.kafka.image.ClusterImage diff --git a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala index 522e70732f4fa..a0637a5a4bf2a 100644 --- a/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/server/MetadataVersionIntegrationTest.scala @@ -17,19 +17,16 @@ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterTest, ClusterTests, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.admin.FeatureUpdate.UpgradeType import org.apache.kafka.clients.admin.{FeatureUpdate, UpdateFeaturesOptions} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.common.MetadataVersion import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class MetadataVersionIntegrationTest { @ClusterTests(value = Array( new ClusterTest(types = Array(Type.KRAFT), metadataVersion = MetadataVersion.IBP_3_3_IV0), diff --git a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala index ad47da549ff41..7196f6ed7eea5 100644 --- a/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala +++ b/core/src/test/scala/integration/kafka/server/RaftClusterSnapshotTest.scala @@ -17,9 +17,8 @@ package kafka.server -import org.apache.kafka.common.test.KafkaClusterTestKit -import org.apache.kafka.common.test.TestKitNodes import kafka.utils.TestUtils +import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.utils.BufferSupplier import org.apache.kafka.metadata.MetadataRecordSerde import org.apache.kafka.server.config.KRaftConfigs diff --git a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala index 15fb0ac2a2f72..f71fc37bb2ff7 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractApiVersionsRequestTest.scala @@ -16,7 +16,6 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.clients.NodeApiVersions import org.apache.kafka.common.message.ApiMessageType.ListenerType import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion @@ -24,6 +23,7 @@ import org.apache.kafka.common.message.ApiMessageType import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, RequestUtils} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Utils import org.apache.kafka.server.common.{EligibleLeaderReplicasVersion, GroupVersion, MetadataVersion, TransactionVersion} import org.apache.kafka.test.TestUtils diff --git a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala index 6c882d3877d83..d54e3227f80c4 100644 --- a/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AllocateProducerIdsRequestTest.scala @@ -18,17 +18,14 @@ package unit.kafka.server import kafka.network.SocketServer import kafka.server.{BrokerServer, ControllerServer, IntegrationTestUtils} -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.AllocateProducerIdsRequestData import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.common.ProducerIdsBlock import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class AllocateProducerIdsRequestTest(cluster: ClusterInstance) { diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index eea2d7cc46b22..6e32cfc01f8be 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -17,17 +17,14 @@ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.message.ApiVersionsRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.ApiVersionsRequest +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.server.common.MetadataVersion import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { @ClusterTest(types = Array(Type.KRAFT, Type.CO_KRAFT), serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala b/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala index 8ff20fedcabcd..4f48dedbf9c40 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerMetricNamesTest.scala @@ -17,18 +17,15 @@ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.ClusterTest -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.metrics.KafkaYammerMetrics import org.junit.jupiter.api.AfterEach import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class BrokerMetricNamesTest(cluster: ClusterInstance) { @AfterEach def tearDown(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala index 7d40f34fd9dd9..1bec9f7f4309c 100644 --- a/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BrokerRegistrationRequestTest.scala @@ -17,7 +17,7 @@ package kafka.server -import org.apache.kafka.common.test.api.{ClusterInstance, ClusterTest, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterTest, Type} import org.apache.kafka.clients.ClientResponse import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic import org.apache.kafka.common.message.{BrokerRegistrationRequestData, CreateTopicsRequestData} @@ -26,11 +26,11 @@ import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Time import org.apache.kafka.common.{Node, Uuid} import org.apache.kafka.server.common.{ControllerRequestCompletionHandler, Feature, MetadataVersion, NodeToControllerChannelManager} import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import java.util import java.util.Collections @@ -39,7 +39,6 @@ import java.util.concurrent.{CompletableFuture, TimeUnit, TimeoutException} /** * This test simulates a broker registering with the KRaft quorum under different configurations. */ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class BrokerRegistrationRequestTest { def brokerToControllerChannelManager(clusterInstance: ClusterInstance): NodeToControllerChannelManager = { diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala index a56914c724859..cec9289e69140 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotasRequestTest.scala @@ -20,23 +20,20 @@ package kafka.server import java.net.InetAddress import java.util import java.util.concurrent.{ExecutionException, TimeUnit} -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.ClusterTest -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.admin.{ScramCredentialInfo, ScramMechanism, UserScramCredentialUpsertion} import org.apache.kafka.common.errors.{InvalidRequestException, UnsupportedVersionException} import org.apache.kafka.common.internals.KafkaFutureImpl import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity, ClientQuotaFilter, ClientQuotaFilterComponent} import org.apache.kafka.common.requests.{AlterClientQuotasRequest, AlterClientQuotasResponse, DescribeClientQuotasRequest, DescribeClientQuotasResponse} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.server.config.{QuotaConfig, ZooKeeperInternals} import org.junit.jupiter.api.Assertions._ import org.junit.jupiter.api.Disabled -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ClientQuotasRequestTest(cluster: ClusterInstance) { @ClusterTest def testAlterClientQuotasRequest(): Unit = { diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala index 8753ceb78dc81..f6831ca8e3dd1 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupDescribeRequestTest.scala @@ -16,9 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api._ -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol @@ -28,18 +26,17 @@ import org.apache.kafka.common.message.{ConsumerGroupDescribeRequestData, Consum import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse} import org.apache.kafka.common.resource.ResourceType +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse} -import org.junit.jupiter.api.extension.ExtendWith import java.lang.{Byte => JByte} import java.util.Collections import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1) class ConsumerGroupDescribeRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { diff --git a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala index 23b5589225dd7..6b42c4f566a94 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerGroupHeartbeatRequestTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry} @@ -25,15 +25,14 @@ import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.message.{ConsumerGroupHeartbeatRequestData, ConsumerGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.{GroupConfig, GroupCoordinatorConfig} import org.apache.kafka.server.common.Feature import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertNotEquals, assertNotNull} -import org.junit.jupiter.api.extension.ExtendWith import scala.collection.Map import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults( types = Array(Type.KRAFT), serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala index b5ef943df844e..0007f32714626 100644 --- a/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala +++ b/core/src/test/scala/unit/kafka/server/ConsumerProtocolMigrationTest.scala @@ -20,23 +20,20 @@ import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.message.{JoinGroupResponseData, ListGroupsResponseData, OffsetFetchResponseData, SyncGroupResponseData} -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.{Group, GroupCoordinatorConfig} import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.Timeout -import org.junit.jupiter.api.extension.ExtendWith import java.nio.ByteBuffer import java.util.Collections import scala.jdk.CollectionConverters._ @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class ConsumerProtocolMigrationTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala index 927eeda688906..e8427bc10dbb1 100644 --- a/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala +++ b/core/src/test/scala/unit/kafka/server/ControllerApisTest.scala @@ -21,7 +21,6 @@ import kafka.network.RequestChannel import kafka.raft.RaftManager import kafka.server.QuotaFactory.QuotaManagers import kafka.server.metadata.KRaftMetadataCache -import org.apache.kafka.common.test.MockController import org.apache.kafka.clients.admin.AlterConfigOp import org.apache.kafka.common.Uuid.ZERO_UUID import org.apache.kafka.common.acl.AclOperation @@ -47,6 +46,7 @@ import org.apache.kafka.common.protocol.{ApiKeys, ApiMessage, Errors} import org.apache.kafka.common.requests._ import org.apache.kafka.common.resource.{PatternType, Resource, ResourcePattern, ResourceType} import org.apache.kafka.common.security.auth.{KafkaPrincipal, SecurityProtocol} +import org.apache.kafka.common.test.MockController import org.apache.kafka.common.utils.MockTime import org.apache.kafka.common.{ElectionType, Uuid} import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT diff --git a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala index 0fab872363bd8..f9b9e9c946aa2 100644 --- a/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DeleteGroupsRequestTest.scala @@ -16,17 +16,14 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.DescribeGroupsResponseData.DescribedGroup import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.junit.jupiter.api.Assertions.{assertEquals, fail} -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class DeleteGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala index de8044ce2c113..4f1ba4b9b2ca3 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeGroupsRequestTest.scala @@ -16,19 +16,16 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.DescribeGroupsResponseData.{DescribedGroup, DescribedGroupMember} import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class DescribeGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala index b564bfd739121..2aa8f5a9e2ca0 100644 --- a/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/DescribeQuorumRequestTest.scala @@ -16,19 +16,16 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.DescribeQuorumRequest.singletonRequest import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, DescribeQuorumRequest, DescribeQuorumResponse} +import org.apache.kafka.common.test.ClusterInstance import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ import scala.reflect.ClassTag -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class DescribeQuorumRequestTest(cluster: ClusterInstance) { diff --git a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala index 7db5258b8e4ef..7f1b7a4d5ff35 100644 --- a/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/GroupCoordinatorBaseRequestTest.scala @@ -17,7 +17,6 @@ package kafka.server import kafka.network.SocketServer -import org.apache.kafka.common.test.api.ClusterInstance import kafka.utils.TestUtils import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} @@ -29,6 +28,7 @@ import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AddOffsetsTo import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{AbstractRequest, AbstractResponse, AddOffsetsToTxnRequest, AddOffsetsToTxnResponse, ConsumerGroupDescribeRequest, ConsumerGroupDescribeResponse, ConsumerGroupHeartbeatRequest, ConsumerGroupHeartbeatResponse, DeleteGroupsRequest, DeleteGroupsResponse, DescribeGroupsRequest, DescribeGroupsResponse, EndTxnRequest, EndTxnResponse, HeartbeatRequest, HeartbeatResponse, InitProducerIdRequest, InitProducerIdResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsRequest, ListGroupsResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetDeleteRequest, OffsetDeleteResponse, OffsetFetchRequest, OffsetFetchResponse, ShareGroupDescribeRequest, ShareGroupDescribeResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, SyncGroupRequest, SyncGroupResponse, TxnOffsetCommitRequest, TxnOffsetCommitResponse} import org.apache.kafka.common.serialization.StringSerializer +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.ProducerIdAndEpoch import org.apache.kafka.controller.ControllerRequestContextUtil.ANONYMOUS_CONTEXT import org.junit.jupiter.api.Assertions.{assertEquals, fail} diff --git a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala index 3d1a06e8b3906..332c01aeeb53b 100644 --- a/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/HeartbeatRequestTest.scala @@ -16,23 +16,20 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.SyncGroupRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState -import org.junit.jupiter.api.extension.ExtendWith import java.util.Collections import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class HeartbeatRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala index 4117080a85a8a..f77c2fc1bfadd 100644 --- a/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/JoinGroupRequestTest.scala @@ -16,19 +16,17 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.JoinGroupResponseData.JoinGroupResponseMember import org.apache.kafka.common.message.{JoinGroupResponseData, SyncGroupRequestData} import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import java.util.Collections import scala.concurrent.ExecutionContext.Implicits.global @@ -36,7 +34,6 @@ import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class JoinGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(types = Array(Type.KRAFT), serverProperties = Array( new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), diff --git a/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala index 4cc3f968d2769..4ff454d1d2de2 100644 --- a/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaveGroupRequestTest.scala @@ -18,20 +18,17 @@ package kafka.server import org.apache.kafka.common.Uuid import org.apache.kafka.common.message.LeaveGroupResponseData -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.JoinGroupRequest +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState import org.junit.jupiter.api.Assertions.assertEquals -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), serverProperties = Array( new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1") diff --git a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala index ea22f5c12167a..3961c725ed445 100644 --- a/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ListGroupsRequestTest.scala @@ -16,18 +16,15 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.message.ListGroupsResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.classic.ClassicGroupState import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup.ConsumerGroupState import org.apache.kafka.coordinator.group.{Group, GroupCoordinatorConfig} import org.junit.jupiter.api.Assertions.{assertEquals, fail} -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class ListGroupsRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala index 6b0c5c6d0a5bb..577007123792b 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitRequestTest.scala @@ -16,15 +16,12 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.junit.jupiter.api.Assertions.fail -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class OffsetCommitRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { diff --git a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala index 7b0672916d5e2..0a808f6c868f4 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetDeleteRequestTest.scala @@ -16,15 +16,12 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.junit.jupiter.api.Assertions.fail -import org.junit.jupiter.api.extension.ExtendWith -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class OffsetDeleteRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest( diff --git a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala index 5bcc256d0c042..a504ecdeea0f8 100644 --- a/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetFetchRequestTest.scala @@ -16,20 +16,16 @@ */ package kafka.server - -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.message.OffsetFetchResponseData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.junit.jupiter.api.Assertions.{assertEquals, fail} -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class OffsetFetchRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 70791a4cef05f..3a8042e2590b3 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -19,16 +19,16 @@ package kafka.server import kafka.api.SaslSetup import kafka.security.JaasTestUtils import kafka.server.SaslApiVersionsRequestTest.{kafkaClientSaslMechanism, kafkaServerSaslMechanisms} -import org.apache.kafka.common.test.api.{ClusterTemplate, Type, ClusterTestExtensions, ClusterConfig, ClusterInstance} +import org.apache.kafka.common.test.api.{ClusterConfig, ClusterTemplate, Type} import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.config.internals.BrokerSecurityConfigs import org.apache.kafka.common.message.SaslHandshakeRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse, SaslHandshakeRequest, SaslHandshakeResponse} import org.apache.kafka.common.security.auth.SecurityProtocol +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.network.SocketServerConfigs import org.junit.jupiter.api.Assertions._ -import org.junit.jupiter.api.extension.ExtendWith import org.junit.jupiter.api.{AfterEach, BeforeEach, Disabled} import java.net.Socket @@ -66,7 +66,6 @@ object SaslApiVersionsRequestTest { } @Disabled("TODO: KAFKA-17631 - Convert SaslApiVersionsRequestTest to kraft") -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) class SaslApiVersionsRequestTest(cluster: ClusterInstance) extends AbstractApiVersionsRequestTest(cluster) { private var sasl: SaslSetup = _ diff --git a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala index 73f9fce42e6b1..9cae66ffbff50 100644 --- a/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareFetchAcknowledgeRequestTest.scala @@ -17,15 +17,15 @@ package kafka.server import kafka.utils.TestUtils -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, ClusterTests, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, ClusterTests, Type} import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords import org.apache.kafka.common.message.{ShareAcknowledgeRequestData, ShareAcknowledgeResponseData, ShareFetchRequestData, ShareFetchResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.requests.{ShareAcknowledgeRequest, ShareAcknowledgeResponse, ShareFetchRequest, ShareFetchResponse, ShareRequestMetadata} +import org.apache.kafka.common.test.ClusterInstance import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.{AfterEach, Tag, Timeout} -import org.junit.jupiter.api.extension.ExtendWith import java.util import java.util.Collections @@ -33,7 +33,6 @@ import scala.collection.convert.ImplicitConversions.`list asScalaBuffer` import scala.jdk.CollectionConverters._ @Timeout(1200) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( new ClusterConfigProperty(key = "group.share.persister.class.name", value = "") )) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala index a6a1129d084bf..cf297198e39e2 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupDescribeRequestTest.scala @@ -16,9 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api._ -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.common.GroupState import org.apache.kafka.common.message.ShareGroupDescribeResponseData.DescribedGroup @@ -26,6 +24,7 @@ import org.apache.kafka.common.message.{ShareGroupDescribeRequestData, ShareGrou import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.{ShareGroupDescribeRequest, ShareGroupDescribeResponse} import org.apache.kafka.common.resource.ResourceType +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.Utils import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.modern.share.ShareGroupConfig @@ -33,13 +32,11 @@ import org.apache.kafka.security.authorizer.AclEntry import org.apache.kafka.server.config.ServerConfigs import org.junit.jupiter.api.Assertions.assertEquals import org.junit.jupiter.api.{Tag, Timeout} -import org.junit.jupiter.api.extension.ExtendWith import java.lang.{Byte => JByte} import scala.jdk.CollectionConverters._ @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( new ClusterConfigProperty(key = ShareGroupConfig.SHARE_GROUP_PERSISTER_CLASS_NAME_CONFIG, value = "") )) diff --git a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala index c471b0f279372..07c7b959ab81c 100644 --- a/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ShareGroupHeartbeatRequestTest.scala @@ -16,7 +16,7 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import kafka.utils.TestUtils.waitForAllPartitionsMetadata import org.apache.kafka.clients.admin.{Admin, NewPartitions} @@ -24,14 +24,13 @@ import org.apache.kafka.common.{TopicPartition, Uuid} import org.apache.kafka.common.message.{ShareGroupHeartbeatRequestData, ShareGroupHeartbeatResponseData} import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse} +import org.apache.kafka.common.test.ClusterInstance import org.junit.jupiter.api.Assertions.{assertEquals, assertNotEquals, assertNotNull, assertNull, assertTrue} import org.junit.jupiter.api.{Tag, Timeout} -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters._ @Timeout(120) -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), brokers = 1, serverProperties = Array( new ClusterConfigProperty(key = "group.share.persister.class.name", value = "") )) diff --git a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala index a52947fdfe4a8..3a53fbf144aba 100644 --- a/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SyncGroupRequestTest.scala @@ -16,24 +16,21 @@ */ package kafka.server -import org.apache.kafka.common.test.api.ClusterInstance import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} -import org.apache.kafka.common.test.api.ClusterTestExtensions import kafka.utils.TestUtils import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor import org.apache.kafka.clients.consumer.internals.ConsumerProtocol import org.apache.kafka.common.message.SyncGroupRequestData import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.group.classic.ClassicGroupState -import org.junit.jupiter.api.extension.ExtendWith import java.util.Collections import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration.Duration import scala.concurrent.{Await, Future} -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT)) class SyncGroupRequestTest(cluster: ClusterInstance) extends GroupCoordinatorBaseRequestTest(cluster) { @ClusterTest(serverProperties = Array( diff --git a/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala b/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala index 57277f3dbed49..b2cd44bbd9222 100644 --- a/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/TxnOffsetCommitRequestTest.scala @@ -16,21 +16,20 @@ */ package kafka.server -import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterInstance, ClusterTest, ClusterTestDefaults, ClusterTestExtensions, Type} +import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterTest, ClusterTestDefaults, Type} import kafka.utils.TestUtils import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.errors.UnsupportedVersionException import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests.JoinGroupRequest +import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.utils.ProducerIdAndEpoch import org.apache.kafka.coordinator.group.GroupCoordinatorConfig import org.apache.kafka.coordinator.transaction.TransactionLogConfig import org.junit.jupiter.api.Assertions.{assertThrows, assertTrue, fail} -import org.junit.jupiter.api.extension.ExtendWith import scala.jdk.CollectionConverters.IterableHasAsScala -@ExtendWith(value = Array(classOf[ClusterTestExtensions])) @ClusterTestDefaults(types = Array(Type.KRAFT), serverProperties = Array( new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_PARTITIONS_CONFIG, value = "1"), new ClusterConfigProperty(key = GroupCoordinatorConfig.OFFSETS_TOPIC_REPLICATION_FACTOR_CONFIG, value = "1"), diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala index 08b6bbe7f21d2..df383b25bf6ca 100644 --- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala +++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala @@ -24,12 +24,12 @@ import java.util.Properties import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import kafka.log.LogManager import kafka.server.{BrokerServer, KafkaConfig, ReplicaManager} -import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import kafka.utils.TestUtils import org.apache.kafka.clients.admin.AlterConfigOp.OpType.SET import org.apache.kafka.clients.admin.{Admin, AlterConfigOp, ConfigEntry, NewTopic} import org.apache.kafka.common.config.ConfigResource import org.apache.kafka.common.config.ConfigResource.Type.BROKER +import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes} import org.apache.kafka.common.utils.Exit import org.apache.kafka.coordinator.group.GroupCoordinator import org.apache.kafka.coordinator.share.ShareCoordinator diff --git a/settings.gradle b/settings.gradle index e640ba906256f..2414593c2e58d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -110,8 +110,8 @@ include 'clients', 'tools:tools-api', 'transaction-coordinator', 'trogdor', - 'test-common', - 'test-common:test-common-api', + 'test-common:test-common-internal-api', + 'test-common:test-common-util', 'test-common:test-common-runtime' project(":storage:api").name = "storage-api" diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java index 45838959eddd2..3c7a2a9397f92 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/RemoteLogSegmentLifecycleTest.java @@ -19,10 +19,9 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogMetadataManager; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; @@ -33,8 +32,6 @@ import org.apache.kafka.storage.internals.log.EpochEntry; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -54,7 +51,6 @@ import static org.mockito.Mockito.verify; @ClusterTestDefaults(brokers = 3) -@ExtendWith(value = ClusterTestExtensions.class) public class RemoteLogSegmentLifecycleTest { private final int segSize = 1048576; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java index 6016491a09524..23af9c50fc331 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest.java @@ -23,17 +23,14 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; import org.apache.kafka.server.log.remote.storage.RemoteStorageException; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -52,7 +49,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3) public class TopicBasedRemoteLogMetadataManagerMultipleSubscriptionsTest { private final ClusterInstance clusterInstance; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java index 0026f64581702..3c0f60650e441 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerRestartTest.java @@ -21,23 +21,19 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import static org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManagerConfig.LOG_DIR; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class TopicBasedRemoteLogMetadataManagerRestartTest { private final Time time = Time.SYSTEM; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java index 20d3d78e37f7e..e36e64560f2a0 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/metadata/storage/TopicBasedRemoteLogMetadataManagerTest.java @@ -21,10 +21,9 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentId; import org.apache.kafka.server.log.remote.storage.RemoteLogSegmentMetadata; @@ -32,7 +31,6 @@ import org.apache.kafka.server.log.remote.storage.RemoteStorageException; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; import java.util.Arrays; @@ -50,7 +48,6 @@ import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3) public class TopicBasedRemoteLogMetadataManagerTest { private static final int SEG_SIZE = 1048576; diff --git a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java index 17f80b63af1f3..840417b5b3e3b 100644 --- a/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java +++ b/storage/src/test/java/org/apache/kafka/server/log/remote/storage/RemoteLogMetadataManagerTest.java @@ -19,17 +19,14 @@ import org.apache.kafka.common.TopicIdPartition; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.Uuid; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.server.log.remote.metadata.storage.RemoteLogMetadataManagerTestUtils; import org.apache.kafka.server.log.remote.metadata.storage.RemotePartitionMetadataStore; import org.apache.kafka.server.log.remote.metadata.storage.TopicBasedRemoteLogMetadataManager; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -44,7 +41,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3) public class RemoteLogMetadataManagerTest { private final ClusterInstance clusterInstance; diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/AutoStart.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java similarity index 97% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java index 1edf465cf345a..4726c539e294a 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfig.java @@ -36,10 +36,10 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_SECURITY_PROTOCOL; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; /** * Represents an immutable requested configuration of a Kafka cluster for integration testing. diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterConfigProperty.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterFeature.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterGenerator.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTemplate.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java similarity index 92% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java index 86aba1030d878..5d4a36f834904 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTest.java @@ -30,8 +30,8 @@ import static java.lang.annotation.ElementType.METHOD; import static java.lang.annotation.RetentionPolicy.RUNTIME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME; @Documented @Target({METHOD}) diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestDefaults.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/ClusterTests.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/DetectThreadLeak.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/README.md b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/README.md similarity index 98% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/README.md rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/README.md index 7a3ea14dc6663..10a7b6dcf9a7f 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/README.md +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/README.md @@ -79,7 +79,7 @@ a JUnit extension called `ClusterTestExtensions` which knows how to process thes invocations. Test classes that wish to make use of these annotations need to explicitly register this extension: ```scala -import org.apache.kafka.common.test.api.ClusterTestExtensions +import org.apache.kafka.common.test.junit.ClusterTestExtensions @ExtendWith(value = Array(classOf[ClusterTestExtensions])) class ApiVersionsRequestTest { diff --git a/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/TestKitDefaults.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/TestKitDefaults.java new file mode 100644 index 0000000000000..3374458222970 --- /dev/null +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/TestKitDefaults.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.common.test.api; + +import org.apache.kafka.common.security.auth.SecurityProtocol; + +/** + * Constants used by TestKitNodes and ClusterTest annotation defaults + */ +public class TestKitDefaults { + public static final int CONTROLLER_ID_OFFSET = 3000; + public static final int BROKER_ID_OFFSET = 0; + public static final SecurityProtocol DEFAULT_BROKER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; + public static final String DEFAULT_BROKER_LISTENER_NAME = "EXTERNAL"; + public static final SecurityProtocol DEFAULT_CONTROLLER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; + public static final String DEFAULT_CONTROLLER_LISTENER_NAME = "CONTROLLER"; + + private TestKitDefaults() { + + } +} diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Type.java b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/Type.java similarity index 59% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Type.java rename to test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/Type.java index 807d4f3354a7a..9e028acf18673 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Type.java +++ b/test-common/test-common-internal-api/src/main/java/org/apache/kafka/common/test/api/Type.java @@ -16,25 +16,9 @@ */ package org.apache.kafka.common.test.api; -import org.junit.jupiter.api.extension.TestTemplateInvocationContext; - - /** * The type of cluster config being requested. Used by {@link org.apache.kafka.common.test.api.ClusterConfig} and the test annotations. */ public enum Type { - KRAFT { - @Override - public TestTemplateInvocationContext invocationContexts(String baseDisplayName, ClusterConfig config) { - return new RaftClusterInvocationContext(baseDisplayName, config, false); - } - }, - CO_KRAFT { - @Override - public TestTemplateInvocationContext invocationContexts(String baseDisplayName, ClusterConfig config) { - return new RaftClusterInvocationContext(baseDisplayName, config, true); - } - }; - - public abstract TestTemplateInvocationContext invocationContexts(String baseDisplayName, ClusterConfig config); + KRAFT, CO_KRAFT; } diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java similarity index 91% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java rename to test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java index 555514774793a..0eeb8ede394de 100644 --- a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java +++ b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/ClusterConfigTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.test.TestUtils; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Assertions; @@ -28,6 +27,7 @@ import java.io.File; import java.io.IOException; import java.lang.reflect.Field; +import java.nio.file.Files; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -35,10 +35,10 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_BROKER_SECURITY_PROTOCOL; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_LISTENER_NAME; -import static org.apache.kafka.common.test.TestKitNodes.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_BROKER_SECURITY_PROTOCOL; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME; +import static org.apache.kafka.common.test.api.TestKitDefaults.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; public class ClusterConfigTest { @@ -51,7 +51,8 @@ private static Map fields(ClusterConfig config) { @Test public void testCopy() throws IOException { - File trustStoreFile = TestUtils.tempFile(); + File trustStoreFile = Files.createTempFile("kafka", ".tmp").toFile(); + trustStoreFile.deleteOnExit(); ClusterConfig clusterConfig = ClusterConfig.builder() .setTypes(Collections.singleton(Type.KRAFT)) diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java b/test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java similarity index 100% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java rename to test-common/test-common-internal-api/src/test/java/org/apache/kafka/common/test/api/DetectThreadLeakTest.java diff --git a/test-common/src/main/resources/log4j2.yaml b/test-common/test-common-internal-api/src/test/resources/log4j2.yaml similarity index 100% rename from test-common/src/main/resources/log4j2.yaml rename to test-common/test-common-internal-api/src/test/resources/log4j2.yaml diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java similarity index 98% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java index 29819b2b0ea41..25758611cb1ac 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstance.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/ClusterInstance.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test; import kafka.log.UnifiedLog; import kafka.network.SocketServer; @@ -42,8 +42,9 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.test.JaasUtils; -import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.authorizer.Authorizer; import org.apache.kafka.server.fault.FaultHandlerException; import org.apache.kafka.storage.internals.checkpoint.OffsetCheckpointFile; diff --git a/test-common/src/main/java/org/apache/kafka/common/test/JaasModule.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasModule.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/JaasModule.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasModule.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/JaasUtils.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasUtils.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/JaasUtils.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/JaasUtils.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterTestKit.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/KafkaClusterThreadFactory.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/MockController.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockController.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/MockController.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockController.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/MockFaultHandler.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/PreboundSocketFactoryManager.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/TestKitNode.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNode.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/TestKitNode.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNode.java diff --git a/test-common/src/main/java/org/apache/kafka/common/test/TestKitNodes.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java similarity index 94% rename from test-common/src/main/java/org/apache/kafka/common/test/TestKitNodes.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java index 24aa7680e320b..c1044f8e5b095 100644 --- a/test-common/src/main/java/org/apache/kafka/common/test/TestKitNodes.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestKitNodes.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.Uuid; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.test.api.TestKitDefaults; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; import org.apache.kafka.metadata.properties.MetaProperties; import org.apache.kafka.metadata.properties.MetaPropertiesEnsemble; @@ -44,13 +45,6 @@ @SuppressWarnings("NPathComplexity") public class TestKitNodes { - public static final int CONTROLLER_ID_OFFSET = 3000; - public static final int BROKER_ID_OFFSET = 0; - public static final SecurityProtocol DEFAULT_BROKER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; - public static final String DEFAULT_BROKER_LISTENER_NAME = "EXTERNAL"; - public static final SecurityProtocol DEFAULT_CONTROLLER_SECURITY_PROTOCOL = SecurityProtocol.PLAINTEXT; - public static final String DEFAULT_CONTROLLER_LISTENER_NAME = "CONTROLLER"; - public static class Builder { private boolean combined; private String clusterId; @@ -70,10 +64,10 @@ public Builder(BootstrapMetadata bootstrapMetadata) { } // The broker and controller listener name and SecurityProtocol configurations must // be kept in sync with the default values in ClusterTest. - private ListenerName brokerListenerName = ListenerName.normalised(DEFAULT_BROKER_LISTENER_NAME); - private SecurityProtocol brokerSecurityProtocol = DEFAULT_BROKER_SECURITY_PROTOCOL; - private ListenerName controllerListenerName = ListenerName.normalised(DEFAULT_CONTROLLER_LISTENER_NAME); - private SecurityProtocol controllerSecurityProtocol = DEFAULT_CONTROLLER_SECURITY_PROTOCOL; + private ListenerName brokerListenerName = ListenerName.normalised(TestKitDefaults.DEFAULT_BROKER_LISTENER_NAME); + private SecurityProtocol brokerSecurityProtocol = TestKitDefaults.DEFAULT_BROKER_SECURITY_PROTOCOL; + private ListenerName controllerListenerName = ListenerName.normalised(TestKitDefaults.DEFAULT_CONTROLLER_LISTENER_NAME); + private SecurityProtocol controllerSecurityProtocol = TestKitDefaults.DEFAULT_CONTROLLER_SECURITY_PROTOCOL; public Builder setClusterId(String clusterId) { this.clusterId = clusterId; @@ -170,11 +164,11 @@ public TestKitNodes build() { clusterId = Uuid.randomUuid().toString(); } - int controllerId = combined ? BROKER_ID_OFFSET : BROKER_ID_OFFSET + CONTROLLER_ID_OFFSET; + int controllerId = combined ? TestKitDefaults.BROKER_ID_OFFSET : TestKitDefaults.BROKER_ID_OFFSET + TestKitDefaults.CONTROLLER_ID_OFFSET; List controllerNodeIds = IntStream.range(controllerId, controllerId + numControllerNodes) .boxed() .collect(Collectors.toList()); - List brokerNodeIds = IntStream.range(BROKER_ID_OFFSET, BROKER_ID_OFFSET + numBrokerNodes) + List brokerNodeIds = IntStream.range(TestKitDefaults.BROKER_ID_OFFSET, TestKitDefaults.BROKER_ID_OFFSET + numBrokerNodes) .boxed() .collect(Collectors.toList()); diff --git a/test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestUtils.java similarity index 100% rename from test-common/src/main/java/org/apache/kafka/common/test/TestUtils.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/TestUtils.java diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstanceParameterResolver.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterInstanceParameterResolver.java similarity index 92% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstanceParameterResolver.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterInstanceParameterResolver.java index 6b3c2339d2e1e..6ec08a9107a1b 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterInstanceParameterResolver.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterInstanceParameterResolver.java @@ -15,17 +15,19 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; +import org.apache.kafka.common.test.ClusterInstance; + import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.api.extension.ParameterContext; import org.junit.jupiter.api.extension.ParameterResolver; +import org.junit.platform.commons.util.AnnotationUtils; import java.lang.reflect.Executable; -import static org.junit.platform.commons.util.AnnotationUtils.isAnnotated; /** * This resolver provides an instance of {@link ClusterInstance} to a test invocation. The instance represents the @@ -57,7 +59,7 @@ public boolean supportsParameter(ParameterContext parameterContext, ExtensionCon } else { // If we're injecting into a method, make sure it's a test method and not a lifecycle method Executable parameterizedMethod = parameterContext.getParameter().getDeclaringExecutable(); - return isAnnotated(parameterizedMethod, TestTemplate.class); + return AnnotationUtils.isAnnotated(parameterizedMethod, TestTemplate.class); } } diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestExtensions.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java similarity index 84% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestExtensions.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java index 65cfac0e0804a..a76e700d068ca 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/ClusterTestExtensions.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/ClusterTestExtensions.java @@ -14,9 +14,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.test.api.AutoStart; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterFeature; +import org.apache.kafka.common.test.api.ClusterTemplate; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.ClusterTests; +import org.apache.kafka.common.test.api.DetectThreadLeak; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.util.timer.SystemTimer; @@ -112,6 +122,13 @@ public boolean supportsTestTemplate(ExtensionContext context) { return true; } + private boolean isClusterTest(ExtensionContext context) { + Method method = context.getRequiredTestMethod(); + return method.getDeclaredAnnotation(ClusterTemplate.class) != null || + method.getDeclaredAnnotation(ClusterTest.class) != null || + method.getDeclaredAnnotation(ClusterTests.class) != null; + } + @Override public Stream provideTestTemplateInvocationContexts(ExtensionContext context) { ClusterTestDefaults defaults = getClusterTestDefaults(context.getRequiredTestClass()); @@ -135,30 +152,29 @@ public Stream provideTestTemplateInvocationContex generatedContexts.addAll(processClusterTests(context, clusterTestsAnnot.value(), defaults)); } - if (generatedContexts.isEmpty()) { - throw new IllegalStateException("Please annotate test methods with @ClusterTemplate, @ClusterTest, or " + - "@ClusterTests when using the ClusterTestExtensions provider"); - } - return generatedContexts.stream(); } @Override public void beforeEach(ExtensionContext context) { - DetectThreadLeak detectThreadLeak = DetectThreadLeak.of(thread -> + if (isClusterTest(context)) { + DetectThreadLeak detectThreadLeak = DetectThreadLeak.of(thread -> SKIPPED_THREAD_PREFIX.stream().noneMatch(prefix -> thread.getName().startsWith(prefix))); - getStore(context).put(DETECT_THREAD_LEAK_KEY, detectThreadLeak); + getStore(context).put(DETECT_THREAD_LEAK_KEY, detectThreadLeak); + } } @Override public void afterEach(ExtensionContext context) { - DetectThreadLeak detectThreadLeak = getStore(context).remove(DETECT_THREAD_LEAK_KEY, DetectThreadLeak.class); - if (detectThreadLeak == null) { - return; - } - List threads = detectThreadLeak.newThreads(); - assertTrue(threads.isEmpty(), "Thread leak detected: " + + if (isClusterTest(context)) { + DetectThreadLeak detectThreadLeak = getStore(context).remove(DETECT_THREAD_LEAK_KEY, DetectThreadLeak.class); + if (detectThreadLeak == null) { + return; + } + List threads = detectThreadLeak.newThreads(); + assertTrue(threads.isEmpty(), "Thread leak detected: " + threads.stream().map(Thread::getName).collect(Collectors.joining(", "))); + } } private Store getStore(ExtensionContext context) { @@ -176,6 +192,21 @@ private int getTestRepeatCount() { return count; } + private TestTemplateInvocationContext invocationContextForClusterType( + Type type, + String baseDisplayName, + ClusterConfig config + ) { + switch (type) { + case KRAFT: + return new RaftClusterInvocationContext(baseDisplayName, config, false); + case CO_KRAFT: + return new RaftClusterInvocationContext(baseDisplayName, config, true); + default: + throw new IllegalArgumentException("Unsupported @Type value " + type); + } + } + List processClusterTemplate(ExtensionContext context, ClusterTemplate annot) { if (annot.value().trim().isEmpty()) { throw new IllegalStateException("ClusterTemplate value can't be empty string."); @@ -186,7 +217,7 @@ List processClusterTemplate(ExtensionContext cont List contexts = IntStream.range(0, repeatCount) .mapToObj(__ -> generateClusterConfigurations(context, annot.value()).stream()) .flatMap(Function.identity()) - .flatMap(config -> config.clusterTypes().stream().map(type -> type.invocationContexts(baseDisplayName, config))) + .flatMap(config -> config.clusterTypes().stream().map(type -> invocationContextForClusterType(type, baseDisplayName, config))) .collect(Collectors.toList()); if (contexts.isEmpty()) { @@ -261,7 +292,7 @@ private List processClusterTestInternal( .build(); return Arrays.stream(types) - .map(type -> type.invocationContexts(context.getRequiredTestMethod().getName(), config)) + .map(type -> invocationContextForClusterType(type, context.getRequiredTestMethod().getName(), config)) .collect(Collectors.toList()); } diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/RaftClusterInvocationContext.java b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/RaftClusterInvocationContext.java similarity index 98% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/RaftClusterInvocationContext.java rename to test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/RaftClusterInvocationContext.java index 22a009b394e6e..76bb24a32b46c 100644 --- a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/RaftClusterInvocationContext.java +++ b/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/RaftClusterInvocationContext.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; import kafka.network.SocketServer; import kafka.server.BrokerServer; @@ -22,9 +22,12 @@ import kafka.server.KafkaBroker; import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.KafkaClusterTestKit; import org.apache.kafka.common.test.TestKitNodes; import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.metadata.BrokerState; import org.apache.kafka.metadata.bootstrap.BootstrapMetadata; diff --git a/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.jupiter.api.extension.Extension b/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.jupiter.api.extension.Extension new file mode 100644 index 0000000000000..d7f05be7df0d4 --- /dev/null +++ b/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.jupiter.api.extension.Extension @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.kafka.common.test.junit.ClusterTestExtensions \ No newline at end of file diff --git a/test-common/test-common-api/src/test/resources/log4j2.yaml b/test-common/test-common-runtime/src/main/resources/log4j2.yaml similarity index 100% rename from test-common/test-common-api/src/test/resources/log4j2.yaml rename to test-common/test-common-runtime/src/main/resources/log4j2.yaml diff --git a/test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java similarity index 100% rename from test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterTestKitTest.java diff --git a/test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java similarity index 100% rename from test-common/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/KafkaClusterThreadFactoryTest.java diff --git a/test-common/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java similarity index 100% rename from test-common/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/TestKitNodeTest.java diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsTest.java similarity index 97% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsTest.java index 20d2d3fd696d8..69faeeb161d05 100644 --- a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsTest.java +++ b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientConfig; @@ -40,14 +40,22 @@ import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.JaasUtils; import org.apache.kafka.common.test.TestUtils; +import org.apache.kafka.common.test.api.AutoStart; +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterConfigProperty; +import org.apache.kafka.common.test.api.ClusterTemplate; +import org.apache.kafka.common.test.api.ClusterTest; +import org.apache.kafka.common.test.api.ClusterTestDefaults; +import org.apache.kafka.common.test.api.ClusterTests; +import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.server.common.MetadataVersion; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.extension.ExtendWith; import java.nio.charset.StandardCharsets; import java.time.Duration; @@ -84,7 +92,6 @@ @ClusterConfigProperty(key = "default.key", value = "default.value"), @ClusterConfigProperty(id = 0, key = "queued.max.requests", value = "100"), }) // Set defaults for a few params in @ClusterTest(s) -@ExtendWith(ClusterTestExtensions.class) public class ClusterTestExtensionsTest { private final ClusterInstance clusterInstance; @@ -357,7 +364,7 @@ public void testControllerListenerName(ClusterInstance cluster) throws Execution } ) public void testSaslPlaintext(ClusterInstance clusterInstance) throws CancellationException, ExecutionException, InterruptedException { - Assertions.assertEquals(SecurityProtocol.SASL_PLAINTEXT, clusterInstance.config().brokerSecurityProtocol()); + assertEquals(SecurityProtocol.SASL_PLAINTEXT, clusterInstance.config().brokerSecurityProtocol()); // default ClusterInstance#admin helper with admin credentials try (Admin admin = clusterInstance.admin()) { diff --git a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsUnitTest.java b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsUnitTest.java similarity index 95% rename from test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsUnitTest.java rename to test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsUnitTest.java index bc21379eaa914..819006612c46f 100644 --- a/test-common/test-common-api/src/test/java/org/apache/kafka/common/test/api/ClusterTestExtensionsUnitTest.java +++ b/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/ClusterTestExtensionsUnitTest.java @@ -15,7 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.common.test.api; +package org.apache.kafka.common.test.junit; + +import org.apache.kafka.common.test.api.ClusterConfig; +import org.apache.kafka.common.test.api.ClusterTemplate; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Flaky.java b/test-common/test-common-util/src/main/java/org/apache/kafka/common/test/api/Flaky.java similarity index 100% rename from test-common/test-common-api/src/main/java/org/apache/kafka/common/test/api/Flaky.java rename to test-common/test-common-util/src/main/java/org/apache/kafka/common/test/api/Flaky.java diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java b/test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java similarity index 100% rename from test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java rename to test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilter.java diff --git a/test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java b/test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java similarity index 100% rename from test-common/test-common-runtime/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java rename to test-common/test-common-util/src/main/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilter.java diff --git a/test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter b/test-common/test-common-util/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter similarity index 100% rename from test-common/test-common-runtime/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter rename to test-common/test-common-util/src/main/resources/META-INF/services/org.junit.platform.launcher.PostDiscoveryFilter diff --git a/test-common/test-common-runtime/src/main/resources/junit-platform.properties b/test-common/test-common-util/src/main/resources/junit-platform.properties similarity index 100% rename from test-common/test-common-runtime/src/main/resources/junit-platform.properties rename to test-common/test-common-util/src/main/resources/junit-platform.properties diff --git a/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java b/test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java similarity index 100% rename from test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java rename to test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/AutoQuarantinedTestFilterTest.java diff --git a/test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java b/test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java similarity index 100% rename from test-common/test-common-runtime/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java rename to test-common/test-common-util/src/test/java/org/apache/kafka/common/test/junit/QuarantinedPostDiscoveryFilterTest.java diff --git a/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java index 12f4dab8801c6..090975e6f0771 100644 --- a/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/AclCommandTest.java @@ -25,11 +25,10 @@ import org.apache.kafka.common.resource.ResourcePattern; import org.apache.kafka.common.resource.ResourceType; import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Exit; @@ -40,7 +39,6 @@ import org.apache.logging.log4j.Level; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.File; import java.io.IOException; @@ -91,7 +89,6 @@ @ClusterConfigProperty(key = AUTHORIZER_CLASS_NAME_CONFIG, value = AclCommandTest.STANDARD_AUTHORIZER)} ) -@ExtendWith(ClusterTestExtensions.class) public class AclCommandTest { public static final String STANDARD_AUTHORIZER = "org.apache.kafka.metadata.authorizer.StandardAuthorizer"; private static final String LOCALHOST = "localhost:9092"; diff --git a/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java index 3f20bb4e50201..49534a0ca8245 100644 --- a/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/BrokerApiVersionsCommandTest.java @@ -24,15 +24,12 @@ import org.apache.kafka.common.message.ApiVersionsResponseData.ApiVersion; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.ApiVersionsResponse; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.server.config.ServerConfigs; -import org.junit.jupiter.api.extension.ExtendWith; - import java.util.Arrays; import java.util.Collections; import java.util.Iterator; @@ -43,7 +40,6 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(ClusterTestExtensions.class) @ClusterTestDefaults(serverProperties = { @ClusterConfigProperty(key = ServerConfigs.UNSTABLE_API_VERSIONS_ENABLE_CONFIG, value = "true"), }) diff --git a/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java b/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java index b21795411e269..8e7a77c292583 100644 --- a/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/ClusterToolTest.java @@ -19,13 +19,11 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.common.errors.UnsupportedEndpointTypeException; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.ByteArrayOutputStream; import java.io.PrintStream; @@ -41,7 +39,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class ClusterToolTest { @ClusterTest diff --git a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java index 0dee52602164d..f798ed702a6d1 100644 --- a/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/DeleteRecordsCommandTest.java @@ -24,16 +24,14 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.server.common.AdminCommandFailedException; import org.apache.kafka.server.common.AdminOperationException; import com.fasterxml.jackson.core.JsonProcessingException; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; import java.nio.file.NoSuchFileException; @@ -48,7 +46,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class DeleteRecordsCommandTest { @ClusterTest diff --git a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java index d923e46cf3138..969a081917017 100644 --- a/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/FeatureCommandTest.java @@ -17,9 +17,8 @@ package org.apache.kafka.tools; import org.apache.kafka.clients.admin.MockAdminClient; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.server.common.Feature; import org.apache.kafka.server.common.MetadataVersion; @@ -27,7 +26,6 @@ import net.sourceforge.argparse4j.inf.Namespace; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collections; @@ -46,7 +44,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class FeatureCommandTest { private final List testingFeatures = Arrays.stream(Feature.FEATURES).collect(Collectors.toList()); diff --git a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java index a641758a827cc..0130da54f5b6c 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GetOffsetShellTest.java @@ -28,13 +28,12 @@ import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.server.config.ServerLogConfigs; @@ -43,8 +42,6 @@ import org.apache.kafka.server.log.remote.storage.RemoteLogManagerConfig; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -65,7 +62,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) @ClusterTestDefaults(serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), @ClusterConfigProperty(key = "offsets.topic.replication.factor", value = "1"), diff --git a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java index 0bc71a2b813aa..767891579778a 100644 --- a/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/GroupsCommandTest.java @@ -32,10 +32,9 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.apache.kafka.test.TestUtils; @@ -44,7 +43,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; -import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.List; @@ -63,7 +61,6 @@ import static org.mockito.Mockito.when; @Timeout(value = 60) -@ExtendWith(ClusterTestExtensions.class) public class GroupsCommandTest { private final String bootstrapServer = "localhost:9092"; diff --git a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java index a74681ce91149..88c6332214bf3 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LeaderElectionCommandTest.java @@ -24,14 +24,12 @@ import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.server.common.AdminCommandFailedException; -import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.ArgumentCaptor; import org.mockito.MockedStatic; import org.mockito.Mockito; @@ -61,7 +59,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) @ClusterTestDefaults(brokers = 3, serverProperties = { @ClusterConfigProperty(key = "auto.create.topics.enable", value = "false"), @ClusterConfigProperty(key = "auto.leader.rebalance.enable", value = "false"), diff --git a/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java index a6ac8a199597e..8407c1fe9ffe8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/LogDirsCommandTest.java @@ -23,15 +23,13 @@ import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collections; @@ -49,7 +47,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) public class LogDirsCommandTest { private static final String TOPIC = "test-log-dirs-topic"; diff --git a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java index 484f09ec5ef00..13b05e306491c 100644 --- a/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/MetadataQuorumCommandTest.java @@ -17,15 +17,13 @@ package org.apache.kafka.tools; import org.apache.kafka.clients.admin.AdminClientConfig; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.ClusterTests; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.test.TestUtils; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.File; import java.io.IOException; @@ -38,7 +36,6 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; -@ExtendWith(value = ClusterTestExtensions.class) class MetadataQuorumCommandTest { /** diff --git a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java index bde7f3f2532d5..1fb2a87f8dbc0 100644 --- a/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/TopicCommandTest.java @@ -48,12 +48,11 @@ import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.metadata.LeaderAndIsr; @@ -63,7 +62,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.ArrayList; import java.util.Arrays; @@ -99,7 +97,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@ExtendWith(ClusterTestExtensions.class) public class TopicCommandTest { private final short defaultReplicationFactor = 1; private final int defaultNumPartitions = 1; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java index fa3f401784389..eef6ac4cb4937 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/ConsoleConsumerTest.java @@ -34,9 +34,8 @@ import org.apache.kafka.common.internals.Topic; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.test.api.ClusterInstance; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.Time; import org.apache.kafka.coordinator.group.generated.GroupMetadataKey; import org.apache.kafka.coordinator.group.generated.GroupMetadataKeyJsonConverter; @@ -57,7 +56,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -96,7 +94,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -@ExtendWith(ClusterTestExtensions.class) public class ConsoleConsumerTest { private final String topic = "test-topic"; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java index 271b496873f58..7d552dcbe483e 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteConsumerGroupsTest.java @@ -25,15 +25,13 @@ import org.apache.kafka.common.errors.GroupNotEmptyException; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.test.TestUtils; import org.apache.kafka.tools.ToolsTestUtils; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.HashMap; @@ -66,8 +64,6 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; - -@ExtendWith(value = ClusterTestExtensions.class) public class DeleteConsumerGroupsTest { private static List generator() { diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java index bd621e64d880d..9509f2913f2d9 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DeleteOffsetsConsumerGroupCommandIntegrationTest.java @@ -32,14 +32,12 @@ import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.coordinator.group.GroupCoordinatorConfig; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.Collections; @@ -54,7 +52,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; -@ExtendWith(ClusterTestExtensions.class) public class DeleteOffsetsConsumerGroupCommandIntegrationTest { public static final String TOPIC_PREFIX = "foo."; public static final String GROUP_PREFIX = "test.group."; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java index 76a5a101548d0..e3669804f1bf5 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/DescribeConsumerGroupTest.java @@ -35,10 +35,9 @@ import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Exit; import org.apache.kafka.test.TestUtils; @@ -46,7 +45,6 @@ import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.ArrayList; import java.util.Arrays; @@ -75,7 +73,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.fail; -@ExtendWith(value = ClusterTestExtensions.class) public class DescribeConsumerGroupTest { private static final String TOPIC_PREFIX = "test.topic."; private static final String GROUP_PREFIX = "test.group."; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java index b16bf8fc0bf03..16db6fb93bc33 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ListConsumerGroupTest.java @@ -28,16 +28,14 @@ import org.apache.kafka.common.GroupType; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.test.TestUtils; import org.apache.kafka.tools.ToolsTestUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; import java.util.ArrayList; import java.util.Arrays; @@ -65,7 +63,6 @@ import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; -@ExtendWith(ClusterTestExtensions.class) public class ListConsumerGroupTest { private static final String TOPIC_PREFIX = "test.topic."; private static final String TOPIC_PARTITIONS_GROUP_PREFIX = "test.topic.partitions.group."; diff --git a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java index a49597d638f9a..bcc06301d6351 100644 --- a/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/consumer/group/ResetConsumerGroupOffsetTest.java @@ -29,14 +29,11 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfig; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTemplate; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.test.TestUtils; -import org.junit.jupiter.api.extension.ExtendWith; - import java.io.BufferedWriter; import java.io.File; import java.io.FileWriter; @@ -92,7 +89,6 @@ * - scope=topics+partitions, scenario=to-earliest * - export/import */ -@ExtendWith(value = ClusterTestExtensions.class) public class ResetConsumerGroupOffsetTest { private static final String TOPIC_PREFIX = "foo-"; diff --git a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java index ae174d63c6ae5..1430197216da8 100644 --- a/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java +++ b/tools/src/test/java/org/apache/kafka/tools/reassign/ReassignPartitionsCommandTest.java @@ -40,11 +40,10 @@ import org.apache.kafka.common.config.ConfigResource; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.test.ClusterInstance; import org.apache.kafka.common.test.api.ClusterConfigProperty; -import org.apache.kafka.common.test.api.ClusterInstance; import org.apache.kafka.common.test.api.ClusterTest; import org.apache.kafka.common.test.api.ClusterTestDefaults; -import org.apache.kafka.common.test.api.ClusterTestExtensions; import org.apache.kafka.common.test.api.ClusterTests; import org.apache.kafka.common.test.api.Type; import org.apache.kafka.common.utils.Time; @@ -55,7 +54,6 @@ import com.fasterxml.jackson.core.JsonProcessingException; import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.util.AbstractMap; @@ -112,7 +110,6 @@ @ClusterConfigProperty(id = 3, key = "broker.rack", value = "rack1"), @ClusterConfigProperty(id = 4, key = "broker.rack", value = "rack1"), }) -@ExtendWith(ClusterTestExtensions.class) public class ReassignPartitionsCommandTest { private final ClusterInstance clusterInstance; private final Map> unthrottledBrokerConfigs = IntStream