KAFKA-14108: Ensure both JUnit 4 and JUnit 5 tests run (#12441)

When the migration of the Streams project to JUnit 5 started with PR #12285, we discovered that the migrated tests were not run by the PR builds. This PR ensures that Streams' tests that are written in JUnit 4 and JUnit 5 are run in the PR builds.

Co-authored-by: Divij Vaidya <diviv@amazon.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>, Bruno Cadonna <cadonna@apache.org>
This commit is contained in:
Christo Lolov 2022-07-29 16:21:25 +01:00 committed by GitHub
parent 5f7c99dd77
commit 54af64c33a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 56 additions and 50 deletions

View File

@ -323,7 +323,7 @@ subprojects {
} }
// Remove the relevant project name once it's converted to JUnit 5 // Remove the relevant project name once it's converted to JUnit 5
def shouldUseJUnit5 = !(["runtime", "streams"].contains(it.project.name)) def shouldUseJUnit5 = !(["runtime"].contains(it.project.name))
def testLoggingEvents = ["passed", "skipped", "failed"] def testLoggingEvents = ["passed", "skipped", "failed"]
def testShowStandardStreams = false def testShowStandardStreams = false
@ -466,8 +466,18 @@ subprojects {
exclude testsToExclude exclude testsToExclude
if (shouldUseJUnit5) { if (shouldUseJUnit5) {
if (project.name == 'streams') {
useJUnitPlatform { useJUnitPlatform {
includeTags "integration" includeTags "integration"
includeTags "org.apache.kafka.test.IntegrationTest"
// Both engines are needed to run JUnit 4 tests alongside JUnit 5 tests.
// junit-vintage (JUnit 4) can be removed once the JUnit 4 migration is complete.
includeEngines "junit-vintage", "junit-jupiter"
}
} else {
useJUnitPlatform {
includeTags "integration"
}
} }
} else { } else {
useJUnit { useJUnit {
@ -503,8 +513,18 @@ subprojects {
exclude testsToExclude exclude testsToExclude
if (shouldUseJUnit5) { if (shouldUseJUnit5) {
if (project.name == 'streams') {
useJUnitPlatform { useJUnitPlatform {
excludeTags "integration" excludeTags "integration"
excludeTags "org.apache.kafka.test.IntegrationTest"
// Both engines are needed to run JUnit 4 tests alongside JUnit 5 tests.
// junit-vintage (JUnit 4) can be removed once the JUnit 4 migration is complete.
includeEngines "junit-vintage", "junit-jupiter"
}
} else {
useJUnitPlatform {
excludeTags "integration"
}
} }
} else { } else {
useJUnit { useJUnit {
@ -1834,11 +1854,12 @@ project(':streams') {
// testCompileOnly prevents streams from exporting a dependency on test-utils, which would cause a dependency cycle // testCompileOnly prevents streams from exporting a dependency on test-utils, which would cause a dependency cycle
testCompileOnly project(':streams:test-utils') testCompileOnly project(':streams:test-utils')
testImplementation project(':clients').sourceSets.test.output testImplementation project(':clients').sourceSets.test.output
testImplementation project(':core') testImplementation project(':core')
testImplementation project(':core').sourceSets.test.output testImplementation project(':core').sourceSets.test.output
testImplementation libs.log4j testImplementation libs.log4j
testImplementation libs.junitJupiterApi testImplementation libs.junitJupiter
testImplementation libs.junitVintageEngine testImplementation libs.junitVintageEngine
testImplementation libs.easymock testImplementation libs.easymock
testImplementation libs.powermockJunit4 testImplementation libs.powermockJunit4

View File

@ -32,13 +32,12 @@ import org.apache.kafka.streams.kstream.Transformer;
import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.processor.PunctuationType; import org.apache.kafka.streams.processor.PunctuationType;
import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender; import org.apache.kafka.streams.processor.internals.testutil.LogCaptureAppender;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -75,7 +74,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class AdjustStreamThreadCountTest { public class AdjustStreamThreadCountTest {
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);

View File

@ -30,13 +30,12 @@ import org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThr
import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.StreamsTestUtils; import org.apache.kafka.test.StreamsTestUtils;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -52,7 +51,7 @@ import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class EmitOnChangeIntegrationTest { public class EmitOnChangeIntegrationTest {
private static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); private static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);

View File

@ -38,13 +38,12 @@ import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Consumed;
import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStream;
import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.StreamsTestUtils; import org.apache.kafka.test.StreamsTestUtils;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -64,7 +63,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.fail; import static org.junit.jupiter.api.Assertions.fail;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class FineGrainedAutoResetIntegrationTest { public class FineGrainedAutoResetIntegrationTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;
private static final String DEFAULT_OUTPUT_TOPIC = "outputTopic"; private static final String DEFAULT_OUTPUT_TOPIC = "outputTopic";

View File

@ -41,14 +41,13 @@ import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore;
import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.ValueAndTimestamp;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockApiProcessorSupplier;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -68,7 +67,7 @@ import static org.hamcrest.core.IsEqual.equalTo;
import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotNull;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class GlobalKTableIntegrationTest { public class GlobalKTableIntegrationTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;

View File

@ -36,14 +36,13 @@ import org.apache.kafka.streams.processor.api.Record;
import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockApiProcessorSupplier; import org.apache.kafka.test.MockApiProcessorSupplier;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -70,7 +69,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
* an exception would be thrown as the GlobalStreamThread closes all global stores on closing. * an exception would be thrown as the GlobalStreamThread closes all global stores on closing.
*/ */
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class GlobalThreadShutDownOrderTest { public class GlobalThreadShutDownOrderTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;
private static final Properties BROKER_CONFIG; private static final Properties BROKER_CONFIG;

View File

@ -40,12 +40,11 @@ import org.apache.kafka.streams.processor.internals.assignment.AssignorConfigura
import org.apache.kafka.streams.processor.internals.assignment.HighAvailabilityTaskAssignor; import org.apache.kafka.streams.processor.internals.assignment.HighAvailabilityTaskAssignor;
import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.NoRetryException; import org.apache.kafka.test.NoRetryException;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -73,7 +72,7 @@ import static org.hamcrest.MatcherAssert.assertThat;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class HighAvailabilityTaskAssignorIntegrationTest { public class HighAvailabilityTaskAssignorIntegrationTest {
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);

View File

@ -53,13 +53,12 @@ import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.ValueAndTimestamp; import org.apache.kafka.streams.state.ValueAndTimestamp;
import org.apache.kafka.streams.state.internals.StoreQueryUtils; import org.apache.kafka.streams.state.internals.StoreQueryUtils;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -88,7 +87,7 @@ import static org.hamcrest.Matchers.matchesPattern;
import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertThrows;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class IQv2IntegrationTest { public class IQv2IntegrationTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;
public static final Duration WINDOW_SIZE = Duration.ofMinutes(5); public static final Duration WINDOW_SIZE = Duration.ofMinutes(5);

View File

@ -41,14 +41,13 @@ import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.ProcessorStateManager;
import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStore;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockMapper; import org.apache.kafka.test.MockMapper;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -74,7 +73,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
*/ */
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class InternalTopicIntegrationTest { public class InternalTopicIntegrationTest {
public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1);

View File

@ -40,14 +40,13 @@ import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.Produced;
import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.Reducer;
import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.TimeWindows;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockMapper; import org.apache.kafka.test.MockMapper;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -65,7 +64,7 @@ import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.sa
* by virtue of having a large commit interval * by virtue of having a large commit interval
*/ */
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
public class KStreamAggregationDedupIntegrationTest { public class KStreamAggregationDedupIntegrationTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;

View File

@ -60,14 +60,13 @@ import org.apache.kafka.streams.processor.ProcessorContext;
import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueIterator;
import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.QueryableStoreTypes;
import org.apache.kafka.streams.state.ReadOnlySessionStore; import org.apache.kafka.streams.state.ReadOnlySessionStore;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.MockMapper; import org.apache.kafka.test.MockMapper;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -100,7 +99,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
@SuppressWarnings({"unchecked", "deprecation"}) @SuppressWarnings({"unchecked", "deprecation"})
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class KStreamAggregationIntegrationTest { public class KStreamAggregationIntegrationTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;

View File

@ -51,20 +51,19 @@ import org.apache.kafka.streams.kstream.TableJoined;
import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueJoiner;
import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.utils.UniqueTopicSerdeScope; import org.apache.kafka.streams.utils.UniqueTopicSerdeScope;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import kafka.utils.MockTime; import kafka.utils.MockTime;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest { public class KTableKTableForeignKeyInnerJoinCustomPartitionerIntegrationTest {
private final static int NUM_BROKERS = 1; private final static int NUM_BROKERS = 1;

View File

@ -33,19 +33,17 @@ import org.apache.kafka.streams.kstream.KTable;
import org.apache.kafka.streams.kstream.Materialized; import org.apache.kafka.streams.kstream.Materialized;
import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.processor.StateRestoreListener;
import org.apache.kafka.streams.processor.WallclockTimestampExtractor; import org.apache.kafka.streams.processor.WallclockTimestampExtractor;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.Method;
import java.time.Duration; import java.time.Duration;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
@ -55,7 +53,7 @@ import java.util.Properties;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class KTableSourceTopicRestartIntegrationTest { public class KTableSourceTopicRestartIntegrationTest {
private static final int NUM_BROKERS = 3; private static final int NUM_BROKERS = 3;
private static final String SOURCE_TOPIC = "source-topic"; private static final String SOURCE_TOPIC = "source-topic";
@ -100,7 +98,7 @@ public class KTableSourceTopicRestartIntegrationTest {
@BeforeEach @BeforeEach
public void before(final TestInfo testInfo) throws Exception { public void before(final TestInfo testInfo) throws Exception {
sourceTopic = SOURCE_TOPIC + "-" + testInfo.getTestMethod().map(Method::getName); sourceTopic = SOURCE_TOPIC + "-" + IntegrationTestUtils.safeUniqueTestName(getClass(), testInfo);
CLUSTER.createTopic(sourceTopic); CLUSTER.createTopic(sourceTopic);
STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, IntegrationTestUtils.safeUniqueTestName(getClass(), testInfo)); STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, IntegrationTestUtils.safeUniqueTestName(getClass(), testInfo));

View File

@ -31,12 +31,11 @@ import org.apache.kafka.streams.processor.TaskId;
import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.KeyValueStore;
import org.apache.kafka.streams.state.StoreBuilder; import org.apache.kafka.streams.state.StoreBuilder;
import org.apache.kafka.streams.state.Stores; import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -60,7 +59,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class RackAwarenessIntegrationTest { public class RackAwarenessIntegrationTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;

View File

@ -53,14 +53,13 @@ import org.apache.kafka.streams.state.Stores;
import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore;
import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder; import org.apache.kafka.streams.state.internals.KeyValueStoreBuilder;
import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.streams.state.internals.OffsetCheckpoint;
import org.apache.kafka.test.IntegrationTest;
import org.apache.kafka.test.TestUtils; import org.apache.kafka.test.TestUtils;
import org.hamcrest.CoreMatchers; import org.hamcrest.CoreMatchers;
import org.junit.experimental.categories.Category;
import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.TestInfo; import org.junit.jupiter.api.TestInfo;
import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.Timeout;
@ -89,7 +88,7 @@ import static org.hamcrest.core.IsEqual.equalTo;
import static org.junit.jupiter.api.Assertions.assertTrue; import static org.junit.jupiter.api.Assertions.assertTrue;
@Timeout(600) @Timeout(600)
@Category(IntegrationTest.class) @Tag("integration")
public class RestoreIntegrationTest { public class RestoreIntegrationTest {
private static final int NUM_BROKERS = 1; private static final int NUM_BROKERS = 1;