From 518d053fe1940fe6762beb20e82a6c1f43f13cde Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 09:32:42 +0800 Subject: [PATCH 1/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../sink/KafkaCommittableSerializerTest.java | 11 +-- .../kafka/sink/KafkaCommitterTest.java | 8 +- ...aRecordSerializationSchemaBuilderTest.java | 40 ++++----- .../kafka/sink/KafkaSinkBuilderTest.java | 4 +- .../connector/kafka/sink/KafkaSinkITCase.java | 61 +++++++------- .../kafka/sink/KafkaTransactionLogITCase.java | 19 +++-- .../kafka/sink/KafkaWriterITCase.java | 12 +-- .../sink/KafkaWriterStateSerializerTest.java | 8 +- .../kafka/sink/TransactionIdFactoryTest.java | 9 ++- .../sink/TransactionToAbortCheckerTest.java | 10 +-- .../kafka/source/KafkaSourceBuilderTest.java | 16 ++-- .../kafka/source/KafkaSourceITCase.java | 10 +-- .../kafka/source/KafkaSourceLegacyITCase.java | 54 ++++++------- .../enumerator/KafkaEnumeratorTest.java | 50 ++++++------ .../KafkaSourceEnumStateSerializerTest.java | 8 +- .../initializer/OffsetsInitializerTest.java | 39 +++++---- .../subscriber/KafkaSubscriberTest.java | 26 +++--- .../metrics/KafkaSourceReaderMetricsTest.java | 10 +-- .../reader/KafkaPartitionSplitReaderTest.java | 14 ++-- .../source/reader/KafkaSourceReaderTest.java | 6 +- .../KafkaRecordDeserializationSchemaTest.java | 14 ++-- .../KafkaPartitionSplitSerializerTest.java | 2 +- .../kafka/FlinkFixedPartitionerTest.java | 10 +-- .../FlinkKafkaConsumerBaseMigrationTest.java | 23 +++--- .../kafka/FlinkKafkaConsumerBaseTest.java | 48 +++++------ .../kafka/FlinkKafkaConsumerITCase.java | 8 +- .../FlinkKafkaInternalProducerITCase.java | 70 ++++++++++------ .../kafka/FlinkKafkaProducerBaseTest.java | 42 ++++++---- .../kafka/FlinkKafkaProducerITCase.java | 44 +++++----- ...inkKafkaProducerMigrationOperatorTest.java | 10 +-- .../FlinkKafkaProducerMigrationTest.java | 9 ++- .../kafka/FlinkKafkaProducerTest.java | 16 ++-- ...JSONKeyValueDeserializationSchemaTest.java | 12 +-- .../kafka/KafkaConsumerTestBase.java | 6 +- .../connectors/kafka/KafkaITCase.java | 81 ++++++++++++------- .../kafka/KafkaMigrationTestBase.java | 22 ++--- .../kafka/KafkaProducerAtLeastOnceITCase.java | 8 +- .../kafka/KafkaProducerExactlyOnceITCase.java | 12 +-- .../kafka/KafkaProducerTestBase.java | 8 +- .../kafka/KafkaShortRetentionTestBase.java | 21 ++--- .../connectors/kafka/KafkaTestBase.java | 23 +++--- .../kafka/KafkaTestBaseWithFlink.java | 4 +- .../kafka/internals/AbstractFetcherTest.java | 10 +-- .../AbstractFetcherWatermarksTest.java | 26 +++--- .../AbstractPartitionDiscovererTest.java | 21 ++--- .../internals/ClosableBlockingQueueTest.java | 22 ++--- .../internals/KafkaTopicPartitionTest.java | 6 +- .../internals/KafkaTopicsDescriptorTest.java | 12 +-- .../KafkaShuffleExactlyOnceITCase.java | 23 +++--- .../kafka/shuffle/KafkaShuffleITCase.java | 33 ++++---- .../kafka/shuffle/KafkaShuffleTestBase.java | 8 +- .../table/KafkaChangelogTableITCase.java | 16 ++-- .../table/KafkaConnectorOptionsUtilTest.java | 14 ++-- .../table/KafkaDynamicTableFactoryTest.java | 54 ++++++------- .../kafka/table/KafkaTableITCase.java | 48 +++++------ .../kafka/table/KafkaTableTestBase.java | 16 ++-- .../kafka/table/ReducingUpsertWriterTest.java | 15 ++-- .../UpsertKafkaDynamicTableFactoryTest.java | 72 +++++++---------- .../kafka/table/UpsertKafkaTableITCase.java | 35 ++++---- 59 files changed, 707 insertions(+), 632 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializerTest.java index 9f1e8126e..069afdb86 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommittableSerializerTest.java @@ -17,9 +17,9 @@ package org.apache.flink.connector.kafka.sink; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; +import org.apache.flink.util.TestLoggerExtension; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; @@ -29,12 +29,13 @@ * Tests for serializing and deserialzing {@link KafkaCommittable} with {@link * KafkaCommittableSerializer}. */ -public class KafkaCommittableSerializerTest extends TestLogger { +@ExtendWith({TestLoggerExtension.class}) +class KafkaCommittableSerializerTest { private static final KafkaCommittableSerializer SERIALIZER = new KafkaCommittableSerializer(); @Test - public void testCommittableSerDe() throws IOException { + void testCommittableSerDe() throws IOException { final String transactionalId = "test-id"; final short epoch = 5; final KafkaCommittable committable = new KafkaCommittable(1L, epoch, transactionalId, null); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java index ea9d893ed..ef6b3068e 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaCommitterTest.java @@ -36,7 +36,7 @@ /** Tests for {@link KafkaCommitter}. */ @ExtendWith({TestLoggerExtension.class}) -public class KafkaCommitterTest { +class KafkaCommitterTest { private static final int PRODUCER_ID = 0; private static final short EPOCH = 0; @@ -44,7 +44,7 @@ public class KafkaCommitterTest { /** Causes a network error by inactive broker and tests that a retry will happen. */ @Test - public void testRetryCommittableOnRetriableError() throws IOException, InterruptedException { + void testRetryCommittableOnRetriableError() throws IOException, InterruptedException { Properties properties = getProperties(); try (final KafkaCommitter committer = new KafkaCommitter(properties); FlinkKafkaInternalProducer producer = @@ -66,7 +66,7 @@ public void testRetryCommittableOnRetriableError() throws IOException, Interrupt } @Test - public void testFailJobOnUnknownFatalError() throws IOException, InterruptedException { + void testFailJobOnUnknownFatalError() throws IOException, InterruptedException { Properties properties = getProperties(); try (final KafkaCommitter committer = new KafkaCommitter(properties); FlinkKafkaInternalProducer producer = @@ -87,7 +87,7 @@ public void testFailJobOnUnknownFatalError() throws IOException, InterruptedExce } @Test - public void testKafkaCommitterClosesProducer() throws IOException, InterruptedException { + void testKafkaCommitterClosesProducer() throws IOException, InterruptedException { Properties properties = getProperties(); FlinkKafkaInternalProducer producer = new FlinkKafkaInternalProducer(properties, TRANSACTIONAL_ID) { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java index 701f9c8aa..66c6c2bd6 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaRecordSerializationSchemaBuilderTest.java @@ -21,8 +21,8 @@ import org.apache.flink.api.common.serialization.SimpleStringSchema; import org.apache.flink.connector.testutils.formats.DummyInitializationContext; import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner; -import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.header.Header; @@ -31,8 +31,9 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.nio.charset.StandardCharsets; import java.util.Arrays; @@ -47,7 +48,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link KafkaRecordSerializationSchemaBuilder}. */ -public class KafkaRecordSerializationSchemaBuilderTest extends TestLogger { +@ExtendWith({TestLoggerExtension.class}) +class KafkaRecordSerializationSchemaBuilderTest { private static final String DEFAULT_TOPIC = "test"; @@ -55,25 +57,25 @@ public class KafkaRecordSerializationSchemaBuilderTest extends TestLogger { private static Map configuration; private static boolean isKeySerializer; - @Before - public void setUp() { + @BeforeEach + void setUp() { configurableConfiguration = new HashMap<>(); configuration = new HashMap<>(); isKeySerializer = false; } @Test - public void testDoNotAllowMultipleKeySerializer() { + void testDoNotAllowMultipleKeySerializer() { assertOnlyOneSerializerAllowed(keySerializationSetter()); } @Test - public void testDoNotAllowMultipleValueSerializer() { + void testDoNotAllowMultipleValueSerializer() { assertOnlyOneSerializerAllowed(valueSerializationSetter()); } @Test - public void testDoNotAllowMultipleTopicSelector() { + void testDoNotAllowMultipleTopicSelector() { assertThatThrownBy( () -> KafkaRecordSerializationSchema.builder() @@ -89,7 +91,7 @@ public void testDoNotAllowMultipleTopicSelector() { } @Test - public void testExpectTopicSelector() { + void testExpectTopicSelector() { assertThatThrownBy( KafkaRecordSerializationSchema.builder() .setValueSerializationSchema(new SimpleStringSchema()) @@ -98,13 +100,13 @@ public void testExpectTopicSelector() { } @Test - public void testExpectValueSerializer() { + void testExpectValueSerializer() { assertThatThrownBy(KafkaRecordSerializationSchema.builder().setTopic(DEFAULT_TOPIC)::build) .isInstanceOf(IllegalStateException.class); } @Test - public void testSerializeRecordWithTopicSelector() { + void testSerializeRecordWithTopicSelector() { final TopicSelector topicSelector = (e) -> { if (e.equals("a")) { @@ -129,7 +131,7 @@ public void testSerializeRecordWithTopicSelector() { } @Test - public void testSerializeRecordWithPartitioner() throws Exception { + void testSerializeRecordWithPartitioner() throws Exception { AtomicBoolean opened = new AtomicBoolean(false); final int partition = 5; final FlinkKafkaPartitioner partitioner = @@ -148,7 +150,7 @@ public void testSerializeRecordWithPartitioner() throws Exception { } @Test - public void testSerializeRecordWithHeaderProvider() throws Exception { + void testSerializeRecordWithHeaderProvider() throws Exception { final HeaderProvider headerProvider = (ignored) -> new RecordHeaders( @@ -169,7 +171,7 @@ public void testSerializeRecordWithHeaderProvider() throws Exception { } @Test - public void testSerializeRecordWithKey() { + void testSerializeRecordWithKey() { final SerializationSchema serializationSchema = new SimpleStringSchema(); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() @@ -184,7 +186,7 @@ public void testSerializeRecordWithKey() { } @Test - public void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception { + void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception { final Map config = Collections.singletonMap("simpleKey", "simpleValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() @@ -201,7 +203,7 @@ public void testKafkaKeySerializerWrapperWithoutConfigurable() throws Exception } @Test - public void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exception { + void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exception { final Map config = Collections.singletonMap("simpleKey", "simpleValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() @@ -215,7 +217,7 @@ public void testKafkaValueSerializerWrapperWithoutConfigurable() throws Exceptio } @Test - public void testSerializeRecordWithKafkaSerializer() throws Exception { + void testSerializeRecordWithKafkaSerializer() throws Exception { final Map config = Collections.singletonMap("configKey", "configValue"); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() @@ -231,7 +233,7 @@ public void testSerializeRecordWithKafkaSerializer() throws Exception { } @Test - public void testSerializeRecordWithTimestamp() { + void testSerializeRecordWithTimestamp() { final SerializationSchema serializationSchema = new SimpleStringSchema(); final KafkaRecordSerializationSchema schema = KafkaRecordSerializationSchema.builder() diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java index eeecc84df..70f4c3474 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkBuilderTest.java @@ -41,7 +41,7 @@ public class KafkaSinkBuilderTest extends TestLogger { }; @Test - public void testPropertyHandling() { + void testPropertyHandling() { validateProducerConfig( getBasicBuilder(), p -> { @@ -78,7 +78,7 @@ public void testPropertyHandling() { } @Test - public void testBootstrapServerSetting() { + void testBootstrapServerSetting() { Properties testConf1 = new Properties(); testConf1.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "testServer"); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java index fda9d6fa3..bcbd7c809 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java @@ -57,24 +57,27 @@ import org.apache.flink.test.util.TestUtils; import org.apache.flink.testutils.junit.SharedObjects; import org.apache.flink.testutils.junit.SharedReference; +import org.apache.flink.testutils.junit.utils.TempDirUtils; import org.apache.flink.util.DockerImageVersions; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.CreateTopicsResult; import org.apache.kafka.clients.admin.DeleteTopicsResult; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Nested; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.extension.ExtendWith; +import org.testcontainers.junit.jupiter.Container; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; @@ -86,6 +89,7 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -107,7 +111,8 @@ import static org.assertj.core.api.Assertions.fail; /** Tests for using KafkaSink writing to a Kafka cluster. */ -public class KafkaSinkITCase extends TestLogger { +@ExtendWith({TestLoggerExtension.class}) +class KafkaSinkITCase { private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; @@ -122,19 +127,19 @@ public class KafkaSinkITCase extends TestLogger { private SharedReference failed; private SharedReference lastCheckpointedRecord; - @ClassRule + @Container public static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KAFKA, LOG) .withEmbeddedZookeeper() .withNetwork(NETWORK) .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); - @Rule public final SharedObjects sharedObjects = SharedObjects.create(); + @RegisterExtension public final SharedObjects sharedObjects = SharedObjects.create(); - @Rule public final TemporaryFolder temp = new TemporaryFolder(); + @TempDir public Path temp; - @BeforeClass - public static void setupAdmin() { + @BeforeAll + static void setupAdmin() { Map properties = new HashMap<>(); properties.put( CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, @@ -142,12 +147,12 @@ public static void setupAdmin() { admin = AdminClient.create(properties); } - @AfterClass - public static void teardownAdmin() { + @AfterAll + static void teardownAdmin() { admin.close(); } - @Before + @BeforeEach public void setUp() throws ExecutionException, InterruptedException, TimeoutException { emittedRecordsCount = sharedObjects.add(new AtomicLong()); emittedRecordsWithCheckpoint = sharedObjects.add(new AtomicLong()); @@ -157,8 +162,8 @@ public void setUp() throws ExecutionException, InterruptedException, TimeoutExce createTestTopic(topic, 1, TOPIC_REPLICATION_FACTOR); } - @After - public void tearDown() throws ExecutionException, InterruptedException, TimeoutException { + @AfterEach + void tearDown() throws ExecutionException, InterruptedException, TimeoutException { deleteTestTopic(topic); } @@ -193,22 +198,22 @@ class IntegrationTests extends SinkTestSuiteBase { } @Test - public void testWriteRecordsToKafkaWithAtLeastOnceGuarantee() throws Exception { + void testWriteRecordsToKafkaWithAtLeastOnceGuarantee() throws Exception { writeRecordsToKafka(DeliveryGuarantee.AT_LEAST_ONCE, emittedRecordsCount); } @Test - public void testWriteRecordsToKafkaWithNoneGuarantee() throws Exception { + void testWriteRecordsToKafkaWithNoneGuarantee() throws Exception { writeRecordsToKafka(DeliveryGuarantee.NONE, emittedRecordsCount); } @Test - public void testWriteRecordsToKafkaWithExactlyOnceGuarantee() throws Exception { + void testWriteRecordsToKafkaWithExactlyOnceGuarantee() throws Exception { writeRecordsToKafka(DeliveryGuarantee.EXACTLY_ONCE, emittedRecordsWithCheckpoint); } @Test - public void testRecoveryWithAtLeastOnceGuarantee() throws Exception { + void testRecoveryWithAtLeastOnceGuarantee() throws Exception { testRecoveryWithAssertion( DeliveryGuarantee.AT_LEAST_ONCE, 1, @@ -216,7 +221,7 @@ public void testRecoveryWithAtLeastOnceGuarantee() throws Exception { } @Test - public void testRecoveryWithExactlyOnceGuarantee() throws Exception { + void testRecoveryWithExactlyOnceGuarantee() throws Exception { testRecoveryWithAssertion( DeliveryGuarantee.EXACTLY_ONCE, 1, @@ -229,7 +234,7 @@ public void testRecoveryWithExactlyOnceGuarantee() throws Exception { } @Test - public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Exception { + void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Exception { testRecoveryWithAssertion( DeliveryGuarantee.EXACTLY_ONCE, 2, @@ -242,12 +247,12 @@ public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throw } @Test - public void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Exception { + void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Exception { // Run a first job failing during the async phase of a checkpoint to leave some // lingering transactions final Configuration config = new Configuration(); config.setString(StateBackendOptions.STATE_BACKEND, "filesystem"); - final File checkpointDir = temp.newFolder(); + final File checkpointDir = TempDirUtils.newFolder(temp); config.setString( CheckpointingOptions.CHECKPOINTS_DIRECTORY, checkpointDir.toURI().toString()); config.set( @@ -279,7 +284,7 @@ public void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Excep } @Test - public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exception { + void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exception { // Run a first job opening 5 transactions one per subtask and fail in async checkpoint phase final Configuration config = new Configuration(); config.set(CoreOptions.DEFAULT_PARALLELISM, 5); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java index 84d732dbf..5dea79aec 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaTransactionLogITCase.java @@ -20,18 +20,20 @@ import org.apache.flink.connector.kafka.sink.KafkaTransactionLog.TransactionRecord; import org.apache.flink.util.TestLogger; +import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.IntegerSerializer; -import org.junit.After; -import org.junit.ClassRule; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; +import org.testcontainers.junit.jupiter.Container; import java.util.ArrayList; import java.util.List; @@ -49,25 +51,26 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link KafkaTransactionLog} to retrieve abortable Kafka transactions. */ -public class KafkaTransactionLogITCase extends TestLogger { +@ExtendWith({TestLoggerExtension.class}) +class KafkaTransactionLogITCase { private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class); private static final String TOPIC_NAME = "kafkaTransactionLogTest"; private static final String TRANSACTIONAL_ID_PREFIX = "kafka-log"; - @ClassRule + @Container public static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KAFKA, LOG).withEmbeddedZookeeper(); private final List> openProducers = new ArrayList<>(); - @After - public void tearDown() { + @AfterEach + void tearDown() { openProducers.forEach(Producer::close); } @Test - public void testGetTransactions() { + void testGetTransactions() { committedTransaction(1); abortedTransaction(2); lingeringTransaction(3); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java index 811ffa207..3233026e5 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterITCase.java @@ -96,17 +96,17 @@ public class KafkaWriterITCase { .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); @BeforeAll - public static void beforeAll() { + static void beforeAll() { KAFKA_CONTAINER.start(); } @AfterAll - public static void afterAll() { + static void afterAll() { KAFKA_CONTAINER.stop(); } @BeforeEach - public void setUp(TestInfo testInfo) { + void setUp(TestInfo testInfo) { metricListener = new MetricListener(); timeService = new TriggerTimeService(); topic = testInfo.getDisplayName().replaceAll("\\W", ""); @@ -129,7 +129,7 @@ public void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception } @Test - public void testIncreasingRecordBasedCounters() throws Exception { + void testIncreasingRecordBasedCounters() throws Exception { final OperatorIOMetricGroup operatorIOMetricGroup = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); final InternalSinkWriterMetricGroup metricGroup = @@ -166,7 +166,7 @@ public void testIncreasingRecordBasedCounters() throws Exception { } @Test - public void testCurrentSendTimeMetric() throws Exception { + void testCurrentSendTimeMetric() throws Exception { final InternalSinkWriterMetricGroup metricGroup = InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()); try (final KafkaWriter writer = @@ -328,7 +328,7 @@ private void triggerProducerException(KafkaWriter writer, Properties pr } @Test - public void testMetadataPublisher() throws Exception { + void testMetadataPublisher() throws Exception { List metadataList = new ArrayList<>(); try (final KafkaWriter writer = createWriterWithConfiguration( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializerTest.java index 3df0ea88c..f1987861e 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/KafkaWriterStateSerializerTest.java @@ -17,9 +17,7 @@ package org.apache.flink.connector.kafka.sink; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; @@ -29,12 +27,12 @@ * Tests for serializing and deserialzing {@link KafkaWriterState} with {@link * KafkaWriterStateSerializer}. */ -public class KafkaWriterStateSerializerTest extends TestLogger { +class KafkaWriterStateSerializerTest { private static final KafkaWriterStateSerializer SERIALIZER = new KafkaWriterStateSerializer(); @Test - public void testStateSerDe() throws IOException { + void testStateSerDe() throws IOException { final KafkaWriterState state = new KafkaWriterState("idPrefix"); final byte[] serialized = SERIALIZER.serialize(state); assertThat(SERIALIZER.deserialize(1, serialized)).isEqualTo(state); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java index 982d58e33..2054c420d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionIdFactoryTest.java @@ -19,15 +19,18 @@ import org.apache.flink.util.TestLogger; -import org.junit.Test; +import org.apache.flink.util.TestLoggerExtension; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link TransactionalIdFactory}. */ -public class TransactionIdFactoryTest extends TestLogger { +@ExtendWith(TestLoggerExtension.class) +class TransactionIdFactoryTest { @Test - public void testBuildTransactionalId() { + void testBuildTransactionalId() { final String expected = "prefix-0-2"; assertThat(TransactionalIdFactory.buildTransactionalId("prefix", 0, 2L)) .isEqualTo(expected); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java index 897a85911..a4c1b64a1 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/sink/TransactionToAbortCheckerTest.java @@ -17,9 +17,7 @@ package org.apache.flink.connector.kafka.sink; -import org.apache.flink.util.TestLogger; - -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; @@ -29,13 +27,13 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link TransactionsToAbortChecker}. */ -public class TransactionToAbortCheckerTest extends TestLogger { +class TransactionToAbortCheckerTest { public static final String ABORT = "abort"; public static final String KEEP = "keep"; @Test - public void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset() { + void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset() { Map offsetMapping = new HashMap<>(2); offsetMapping.put(0, 1L); offsetMapping.put(2, 3L); @@ -63,7 +61,7 @@ public void testMustAbortTransactionsWithSameSubtaskIdAndHigherCheckpointOffset( } @Test - public void testMustAbortTransactionsIfLowestCheckpointOffsetIsMinimumOffset() { + void testMustAbortTransactionsIfLowestCheckpointOffsetIsMinimumOffset() { final TransactionsToAbortChecker checker = new TransactionsToAbortChecker(2, Collections.singletonMap(0, 1L), 0); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java index 236e9618f..64bacf28a 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceBuilderTest.java @@ -42,10 +42,10 @@ /** Tests for {@link KafkaSourceBuilder}. */ @ExtendWith(TestLoggerExtension.class) -public class KafkaSourceBuilderTest { +class KafkaSourceBuilderTest { @Test - public void testBuildSourceWithGroupId() { + void testBuildSourceWithGroupId() { final KafkaSource kafkaSource = getBasicBuilder().setGroupId("groupId").build(); // Commit on checkpoint should be enabled by default assertThat( @@ -65,7 +65,7 @@ public void testBuildSourceWithGroupId() { } @Test - public void testBuildSourceWithoutGroupId() { + void testBuildSourceWithoutGroupId() { final KafkaSource kafkaSource = getBasicBuilder().build(); // Commit on checkpoint and auto commit should be disabled because group.id is not specified assertThat( @@ -84,7 +84,7 @@ public void testBuildSourceWithoutGroupId() { } @Test - public void testEnableCommitOnCheckpointWithoutGroupId() { + void testEnableCommitOnCheckpointWithoutGroupId() { assertThatThrownBy( () -> getBasicBuilder() @@ -99,7 +99,7 @@ public void testEnableCommitOnCheckpointWithoutGroupId() { } @Test - public void testEnableAutoCommitWithoutGroupId() { + void testEnableAutoCommitWithoutGroupId() { assertThatThrownBy( () -> getBasicBuilder() @@ -112,7 +112,7 @@ public void testEnableAutoCommitWithoutGroupId() { } @Test - public void testDisableOffsetCommitWithoutGroupId() { + void testDisableOffsetCommitWithoutGroupId() { getBasicBuilder() .setProperty(KafkaSourceOptions.COMMIT_OFFSETS_ON_CHECKPOINT.key(), "false") .build(); @@ -120,7 +120,7 @@ public void testDisableOffsetCommitWithoutGroupId() { } @Test - public void testUsingCommittedOffsetsInitializerWithoutGroupId() { + void testUsingCommittedOffsetsInitializerWithoutGroupId() { // Using OffsetsInitializer#committedOffsets as starting offsets assertThatThrownBy( () -> @@ -158,7 +158,7 @@ public void testUsingCommittedOffsetsInitializerWithoutGroupId() { } @Test - public void testSettingCustomKafkaSubscriber() { + void testSettingCustomKafkaSubscriber() { ExampleCustomSubscriber exampleCustomSubscriber = new ExampleCustomSubscriber(); KafkaSourceBuilder customKafkaSubscriberBuilder = new KafkaSourceBuilder() diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java index e37aefe0e..dac1893a2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java @@ -83,7 +83,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Unite test class for {@link KafkaSource}. */ -public class KafkaSourceITCase { +class KafkaSourceITCase { private static final String TOPIC1 = "topic1"; private static final String TOPIC2 = "topic2"; @@ -168,7 +168,7 @@ public void testBasicRead(boolean enableObjectReuse) throws Exception { } @Test - public void testValueOnlyDeserializer() throws Exception { + void testValueOnlyDeserializer() throws Exception { KafkaSource source = KafkaSource.builder() .setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings) @@ -265,7 +265,7 @@ public void testBasicReadWithoutGroupId(boolean enableObjectReuse) throws Except } @Test - public void testPerPartitionWatermark() throws Throwable { + void testPerPartitionWatermark() throws Throwable { String watermarkTopic = "watermarkTestTopic-" + UUID.randomUUID(); KafkaSourceTestEnv.createTestTopic(watermarkTopic, 2, 1); List> records = @@ -312,7 +312,7 @@ public void processElement( } @Test - public void testConsumingEmptyTopic() throws Throwable { + void testConsumingEmptyTopic() throws Throwable { String emptyTopic = "emptyTopic-" + UUID.randomUUID(); KafkaSourceTestEnv.createTestTopic(emptyTopic, 3, 1); KafkaSource source = @@ -337,7 +337,7 @@ public void testConsumingEmptyTopic() throws Throwable { } @Test - public void testConsumingTopicWithEmptyPartitions() throws Throwable { + void testConsumingTopicWithEmptyPartitions() throws Throwable { String topicWithEmptyPartitions = "topicWithEmptyPartitions-" + UUID.randomUUID(); KafkaSourceTestEnv.createTestTopic( topicWithEmptyPartitions, KafkaSourceTestEnv.NUM_PARTITIONS, 1); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java index 5cc0ddf63..015800df6 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceLegacyITCase.java @@ -23,75 +23,75 @@ import org.apache.flink.streaming.connectors.kafka.KafkaProducerTestBase; import org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; /** * An IT case class that runs all the IT cases of the legacy {@link * org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer} with the new {@link KafkaSource}. */ -public class KafkaSourceLegacyITCase extends KafkaConsumerTestBase { +class KafkaSourceLegacyITCase extends KafkaConsumerTestBase { public KafkaSourceLegacyITCase() throws Exception { super(true); } - @BeforeClass - public static void prepare() throws Exception { + @BeforeAll + protected static void prepare() throws Exception { KafkaProducerTestBase.prepare(); ((KafkaTestEnvironmentImpl) kafkaServer) .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); } @Test - public void testFailOnNoBroker() throws Exception { + void testFailOnNoBroker() throws Exception { runFailOnNoBrokerTest(); } @Test - public void testConcurrentProducerConsumerTopology() throws Exception { + void testConcurrentProducerConsumerTopology() throws Exception { runSimpleConcurrentProducerConsumerTopology(); } @Test - public void testKeyValueSupport() throws Exception { + void testKeyValueSupport() throws Exception { runKeyValueTest(); } // --- canceling / failures --- @Test - public void testCancelingEmptyTopic() throws Exception { + void testCancelingEmptyTopic() throws Exception { runCancelingOnEmptyInputTest(); } @Test - public void testCancelingFullTopic() throws Exception { + void testCancelingFullTopic() throws Exception { runCancelingOnFullInputTest(); } // --- source to partition mappings and exactly once --- @Test - public void testOneToOneSources() throws Exception { + void testOneToOneSources() throws Exception { runOneToOneExactlyOnceTest(); } @Test - public void testOneSourceMultiplePartitions() throws Exception { + void testOneSourceMultiplePartitions() throws Exception { runOneSourceMultiplePartitionsExactlyOnceTest(); } @Test - public void testMultipleSourcesOnePartition() throws Exception { + void testMultipleSourcesOnePartition() throws Exception { runMultipleSourcesOnePartitionExactlyOnceTest(); } // --- broker failure --- @Test - @Ignore("FLINK-28267") + @Disabled("FLINK-28267") public void testBrokerFailure() throws Exception { runBrokerFailureTest(); } @@ -99,66 +99,66 @@ public void testBrokerFailure() throws Exception { // --- special executions --- @Test - public void testBigRecordJob() throws Exception { + void testBigRecordJob() throws Exception { runBigRecordTestTopology(); } @Test - public void testMultipleTopicsWithLegacySerializer() throws Exception { + void testMultipleTopicsWithLegacySerializer() throws Exception { runProduceConsumeMultipleTopics(true); } @Test - public void testMultipleTopicsWithKafkaSerializer() throws Exception { + void testMultipleTopicsWithKafkaSerializer() throws Exception { runProduceConsumeMultipleTopics(false); } @Test - public void testAllDeletes() throws Exception { + void testAllDeletes() throws Exception { runAllDeletesTest(); } // --- startup mode --- @Test - public void testStartFromEarliestOffsets() throws Exception { + void testStartFromEarliestOffsets() throws Exception { runStartFromEarliestOffsets(); } @Test - public void testStartFromLatestOffsets() throws Exception { + void testStartFromLatestOffsets() throws Exception { runStartFromLatestOffsets(); } @Test - public void testStartFromGroupOffsets() throws Exception { + void testStartFromGroupOffsets() throws Exception { runStartFromGroupOffsets(); } @Test - public void testStartFromSpecificOffsets() throws Exception { + void testStartFromSpecificOffsets() throws Exception { runStartFromSpecificOffsets(); } @Test - public void testStartFromTimestamp() throws Exception { + void testStartFromTimestamp() throws Exception { runStartFromTimestamp(); } // --- offset committing --- @Test - public void testCommitOffsetsToKafka() throws Exception { + void testCommitOffsetsToKafka() throws Exception { runCommitOffsetsToKafka(); } @Test - public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception { + void testAutoOffsetRetrievalAndCommitToKafka() throws Exception { runAutoOffsetRetrievalAndCommitToKafka(); } @Test - public void testCollectingSchema() throws Exception { + void testCollectingSchema() throws Exception { runCollectingSchemaTest(); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java index 8b308af16..0681b6385 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaEnumeratorTest.java @@ -35,9 +35,10 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Arrays; @@ -56,7 +57,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Unit tests for {@link KafkaSourceEnumerator}. */ -public class KafkaEnumeratorTest { +class KafkaEnumeratorTest { private static final int NUM_SUBTASKS = 3; private static final String DYNAMIC_TOPIC_NAME = "dynamic_topic"; private static final int NUM_PARTITIONS_DYNAMIC_TOPIC = 4; @@ -75,20 +76,20 @@ public class KafkaEnumeratorTest { private static final boolean INCLUDE_DYNAMIC_TOPIC = true; private static final boolean EXCLUDE_DYNAMIC_TOPIC = false; - @BeforeClass - public static void setup() throws Throwable { + @BeforeAll + static void setup() throws Throwable { KafkaSourceTestEnv.setup(); KafkaSourceTestEnv.setupTopic(TOPIC1, true, true, KafkaSourceTestEnv::getRecordsForTopic); KafkaSourceTestEnv.setupTopic(TOPIC2, true, true, KafkaSourceTestEnv::getRecordsForTopic); } - @AfterClass - public static void tearDown() throws Exception { + @AfterAll + static void tearDown() throws Exception { KafkaSourceTestEnv.tearDown(); } @Test - public void testStartWithDiscoverPartitionsOnce() throws Exception { + void testStartWithDiscoverPartitionsOnce() throws Exception { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -109,7 +110,7 @@ public void testStartWithDiscoverPartitionsOnce() throws Exception { } @Test - public void testStartWithPeriodicPartitionDiscovery() throws Exception { + void testStartWithPeriodicPartitionDiscovery() throws Exception { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -130,7 +131,7 @@ public void testStartWithPeriodicPartitionDiscovery() throws Exception { } @Test - public void testDiscoverPartitionsTriggersAssignments() throws Throwable { + void testDiscoverPartitionsTriggersAssignments() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -155,7 +156,7 @@ public void testDiscoverPartitionsTriggersAssignments() throws Throwable { } @Test - public void testReaderRegistrationTriggersAssignments() throws Throwable { + void testReaderRegistrationTriggersAssignments() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -178,7 +179,7 @@ public void testReaderRegistrationTriggersAssignments() throws Throwable { } @Test - public void testRunWithDiscoverPartitionsOnceToCheckNoMoreSplit() throws Throwable { + void testRunWithDiscoverPartitionsOnceToCheckNoMoreSplit() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -202,7 +203,7 @@ public void testRunWithDiscoverPartitionsOnceToCheckNoMoreSplit() throws Throwab } @Test - public void testRunWithPeriodicPartitionDiscoveryOnceToCheckNoMoreSplit() throws Throwable { + void testRunWithPeriodicPartitionDiscoveryOnceToCheckNoMoreSplit() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -226,7 +227,7 @@ public void testRunWithPeriodicPartitionDiscoveryOnceToCheckNoMoreSplit() throws } @Test - public void testRunWithDiscoverPartitionsOnceWithZeroMsToCheckNoMoreSplit() throws Throwable { + void testRunWithDiscoverPartitionsOnceWithZeroMsToCheckNoMoreSplit() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); // Disable periodic partition discovery @@ -249,8 +250,9 @@ public void testRunWithDiscoverPartitionsOnceWithZeroMsToCheckNoMoreSplit() thro } } - @Test(timeout = 30000L) - public void testDiscoverPartitionsPeriodically() throws Throwable { + @Test + @Timeout(30L) + void testDiscoverPartitionsPeriodically() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -317,7 +319,7 @@ public void testDiscoverPartitionsPeriodically() throws Throwable { } @Test - public void testAddSplitsBack() throws Throwable { + void testAddSplitsBack() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -342,7 +344,7 @@ public void testAddSplitsBack() throws Throwable { } @Test - public void testWorkWithPreexistingAssignments() throws Throwable { + void testWorkWithPreexistingAssignments() throws Throwable { Set preexistingAssignments; try (MockSplitEnumeratorContext context1 = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); @@ -377,7 +379,7 @@ public void testWorkWithPreexistingAssignments() throws Throwable { } @Test - public void testKafkaClientProperties() throws Exception { + void testKafkaClientProperties() throws Exception { Properties properties = new Properties(); String clientIdPrefix = "test-prefix"; Integer defaultTimeoutMs = 99999; @@ -410,7 +412,7 @@ public void testKafkaClientProperties() throws Exception { } @Test - public void testSnapshotState() throws Throwable { + void testSnapshotState() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = createEnumerator(context, false)) { @@ -463,7 +465,7 @@ public void testSnapshotState() throws Throwable { } @Test - public void testPartitionChangeChecking() throws Throwable { + void testPartitionChangeChecking() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = @@ -500,7 +502,7 @@ public void testPartitionChangeChecking() throws Throwable { } @Test - public void testEnablePartitionDiscoveryByDefault() throws Throwable { + void testEnablePartitionDiscoveryByDefault() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); KafkaSourceEnumerator enumerator = createEnumerator(context, new Properties())) { @@ -514,7 +516,7 @@ public void testEnablePartitionDiscoveryByDefault() throws Throwable { } @Test - public void testDisablePartitionDiscovery() throws Throwable { + void testDisablePartitionDiscovery() throws Throwable { Properties props = new Properties(); props.setProperty( KafkaSourceOptions.PARTITION_DISCOVERY_INTERVAL_MS.key(), String.valueOf(0)); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java index 6c172e4a2..fbab252e2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/KafkaSourceEnumStateSerializerTest.java @@ -23,7 +23,7 @@ import org.apache.flink.connector.kafka.source.split.KafkaPartitionSplitSerializer; import org.apache.kafka.common.TopicPartition; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.IOException; import java.util.Collection; @@ -35,7 +35,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link KafkaSourceEnumStateSerializer}. */ -public class KafkaSourceEnumStateSerializerTest { +class KafkaSourceEnumStateSerializerTest { private static final int NUM_READERS = 10; private static final String TOPIC_PREFIX = "topic-"; @@ -43,7 +43,7 @@ public class KafkaSourceEnumStateSerializerTest { private static final long STARTING_OFFSET = KafkaPartitionSplit.EARLIEST_OFFSET; @Test - public void testEnumStateSerde() throws IOException { + void testEnumStateSerde() throws IOException { final KafkaSourceEnumState state = new KafkaSourceEnumState( constructTopicPartitions(0), @@ -63,7 +63,7 @@ public void testEnumStateSerde() throws IOException { } @Test - public void testBackwardCompatibility() throws IOException { + void testBackwardCompatibility() throws IOException { final Set topicPartitions = constructTopicPartitions(0); final Map> splitAssignments = diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java index 46dd61a6f..12da5b5dc 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/initializer/OffsetsInitializerTest.java @@ -25,9 +25,9 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.util.Collections; import java.util.HashMap; @@ -36,16 +36,17 @@ import java.util.stream.Collectors; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Unit tests for {@link OffsetsInitializer}. */ -public class OffsetsInitializerTest { +class OffsetsInitializerTest { private static final String TOPIC = "topic"; private static final String TOPIC2 = "topic2"; private static final String EMPTY_TOPIC3 = "topic3"; private static KafkaSourceEnumerator.PartitionOffsetsRetrieverImpl retriever; - @BeforeClass - public static void setup() throws Throwable { + @BeforeAll + static void setup() throws Throwable { KafkaSourceTestEnv.setup(); KafkaSourceTestEnv.setupTopic(TOPIC, true, true, KafkaSourceTestEnv::getRecordsForTopic); KafkaSourceTestEnv.setupTopic(TOPIC2, false, false, KafkaSourceTestEnv::getRecordsForTopic); @@ -56,14 +57,14 @@ public static void setup() throws Throwable { KafkaSourceTestEnv.getAdminClient(), KafkaSourceTestEnv.GROUP_ID); } - @AfterClass - public static void tearDown() throws Exception { + @AfterAll + static void tearDown() throws Exception { retriever.close(); KafkaSourceTestEnv.tearDown(); } @Test - public void testEarliestOffsetsInitializer() { + void testEarliestOffsetsInitializer() { OffsetsInitializer initializer = OffsetsInitializer.earliest(); List partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC); Map offsets = initializer.getPartitionOffsets(partitions, retriever); @@ -77,7 +78,7 @@ public void testEarliestOffsetsInitializer() { } @Test - public void testLatestOffsetsInitializer() { + void testLatestOffsetsInitializer() { OffsetsInitializer initializer = OffsetsInitializer.latest(); List partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC); Map offsets = initializer.getPartitionOffsets(partitions, retriever); @@ -90,7 +91,7 @@ public void testLatestOffsetsInitializer() { } @Test - public void testCommittedGroupOffsetsInitializer() { + void testCommittedGroupOffsetsInitializer() { OffsetsInitializer initializer = OffsetsInitializer.committedOffsets(); List partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC); Map offsets = initializer.getPartitionOffsets(partitions, retriever); @@ -102,7 +103,7 @@ public void testCommittedGroupOffsetsInitializer() { } @Test - public void testTimestampOffsetsInitializer() { + void testTimestampOffsetsInitializer() { OffsetsInitializer initializer = OffsetsInitializer.timestamp(2001); List partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC); Map offsets = initializer.getPartitionOffsets(partitions, retriever); @@ -115,7 +116,7 @@ public void testTimestampOffsetsInitializer() { } @Test - public void testTimestampOffsetsInitializerForEmptyPartitions() { + void testTimestampOffsetsInitializerForEmptyPartitions() { OffsetsInitializer initializer = OffsetsInitializer.timestamp(2001); List partitions = KafkaSourceTestEnv.getPartitionsForTopic(EMPTY_TOPIC3); Map expectedOffsets = @@ -127,7 +128,7 @@ public void testTimestampOffsetsInitializerForEmptyPartitions() { } @Test - public void testSpecificOffsetsInitializer() { + void testSpecificOffsetsInitializer() { Map specifiedOffsets = new HashMap<>(); List partitions = KafkaSourceTestEnv.getPartitionsForTopic(TOPIC); Map committedOffsets = @@ -161,10 +162,14 @@ public void testSpecificOffsetsInitializer() { } } - @Test(expected = IllegalStateException.class) - public void testSpecifiedOffsetsInitializerWithoutOffsetResetStrategy() { + @Test + void testSpecifiedOffsetsInitializerWithoutOffsetResetStrategy() { OffsetsInitializer initializer = OffsetsInitializer.offsets(Collections.emptyMap(), OffsetResetStrategy.NONE); - initializer.getPartitionOffsets(KafkaSourceTestEnv.getPartitionsForTopic(TOPIC), retriever); + assertThatThrownBy( + () -> + initializer.getPartitionOffsets( + KafkaSourceTestEnv.getPartitionsForTopic(TOPIC), retriever)) + .isInstanceOf(IllegalStateException.class); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java index 258c1c0ab..d83e9e09f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/enumerator/subscriber/KafkaSubscriberTest.java @@ -23,9 +23,9 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; @@ -39,28 +39,28 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Unit tests for {@link KafkaSubscriber}. */ -public class KafkaSubscriberTest { +class KafkaSubscriberTest { private static final String TOPIC1 = "topic1"; private static final String TOPIC2 = "pattern-topic"; private static final TopicPartition NON_EXISTING_TOPIC = new TopicPartition("removed", 0); private static AdminClient adminClient; - @BeforeClass - public static void setup() throws Throwable { + @BeforeAll + static void setup() throws Throwable { KafkaSourceTestEnv.setup(); KafkaSourceTestEnv.createTestTopic(TOPIC1); KafkaSourceTestEnv.createTestTopic(TOPIC2); adminClient = KafkaSourceTestEnv.getAdminClient(); } - @AfterClass - public static void tearDown() throws Exception { + @AfterAll + static void tearDown() throws Exception { adminClient.close(); KafkaSourceTestEnv.tearDown(); } @Test - public void testTopicListSubscriber() { + void testTopicListSubscriber() { List topics = Arrays.asList(TOPIC1, TOPIC2); KafkaSubscriber subscriber = KafkaSubscriber.getTopicListSubscriber(Arrays.asList(TOPIC1, TOPIC2)); @@ -74,7 +74,7 @@ public void testTopicListSubscriber() { } @Test - public void testNonExistingTopic() { + void testNonExistingTopic() { final KafkaSubscriber subscriber = KafkaSubscriber.getTopicListSubscriber( Collections.singletonList(NON_EXISTING_TOPIC.topic())); @@ -85,7 +85,7 @@ public void testNonExistingTopic() { } @Test - public void testTopicPatternSubscriber() { + void testTopicPatternSubscriber() { KafkaSubscriber subscriber = KafkaSubscriber.getTopicPatternSubscriber(Pattern.compile("pattern.*")); final Set subscribedPartitions = @@ -99,7 +99,7 @@ public void testTopicPatternSubscriber() { } @Test - public void testPartitionSetSubscriber() { + void testPartitionSetSubscriber() { List topics = Arrays.asList(TOPIC1, TOPIC2); Set partitions = new HashSet<>(KafkaSourceTestEnv.getPartitionsForTopics(topics)); @@ -114,7 +114,7 @@ public void testPartitionSetSubscriber() { } @Test - public void testNonExistingPartition() { + void testNonExistingPartition() { TopicPartition nonExistingPartition = new TopicPartition(TOPIC1, Integer.MAX_VALUE); final KafkaSubscriber subscriber = KafkaSubscriber.getPartitionSetSubscriber( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetricsTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetricsTest.java index d9dd1d3b0..a14576403 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetricsTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/metrics/KafkaSourceReaderMetricsTest.java @@ -24,7 +24,7 @@ import org.apache.flink.runtime.metrics.groups.InternalSourceReaderMetricGroup; import org.apache.kafka.common.TopicPartition; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.Optional; @@ -42,7 +42,7 @@ public class KafkaSourceReaderMetricsTest { private static final TopicPartition BAR_1 = new TopicPartition("bar", 1); @Test - public void testCurrentOffsetTracking() { + void testCurrentOffsetTracking() { MetricListener metricListener = new MetricListener(); final KafkaSourceReaderMetrics kafkaSourceReaderMetrics = @@ -66,7 +66,7 @@ public void testCurrentOffsetTracking() { } @Test - public void testCommitOffsetTracking() { + void testCommitOffsetTracking() { MetricListener metricListener = new MetricListener(); final KafkaSourceReaderMetrics kafkaSourceReaderMetrics = @@ -101,7 +101,7 @@ public void testCommitOffsetTracking() { } @Test - public void testNonTrackingTopicPartition() { + void testNonTrackingTopicPartition() { MetricListener metricListener = new MetricListener(); final KafkaSourceReaderMetrics kafkaSourceReaderMetrics = new KafkaSourceReaderMetrics( @@ -113,7 +113,7 @@ public void testNonTrackingTopicPartition() { } @Test - public void testFailedCommit() { + void testFailedCommit() { MetricListener metricListener = new MetricListener(); final KafkaSourceReaderMetrics kafkaSourceReaderMetrics = new KafkaSourceReaderMetrics( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java index b592a6917..9103fcddf 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaPartitionSplitReaderTest.java @@ -100,14 +100,14 @@ public static void tearDown() throws Exception { } @Test - public void testHandleSplitChangesAndFetch() throws Exception { + void testHandleSplitChangesAndFetch() throws Exception { KafkaPartitionSplitReader reader = createReader(); assignSplitsAndFetchUntilFinish(reader, 0); assignSplitsAndFetchUntilFinish(reader, 1); } @Test - public void testWakeUp() throws Exception { + void testWakeUp() throws Exception { KafkaPartitionSplitReader reader = createReader(); TopicPartition nonExistingTopicPartition = new TopicPartition("NotExist", 0); assignSplits( @@ -136,7 +136,7 @@ public void testWakeUp() throws Exception { } @Test - public void testWakeupThenAssign() throws IOException { + void testWakeupThenAssign() throws IOException { KafkaPartitionSplitReader reader = createReader(); // Assign splits with records assignSplits(reader, splitsByOwners.get(0)); @@ -154,7 +154,7 @@ public void testWakeupThenAssign() throws IOException { } @Test - public void testNumBytesInCounter() throws Exception { + void testNumBytesInCounter() throws Exception { final OperatorMetricGroup operatorMetricGroup = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup(); final Counter numBytesInCounter = @@ -236,7 +236,7 @@ public void testPendingRecordsGauge(String topicSuffix) throws Throwable { } @Test - public void testAssignEmptySplit() throws Exception { + void testAssignEmptySplit() throws Exception { KafkaPartitionSplitReader reader = createReader(); final KafkaPartitionSplit normalSplit = new KafkaPartitionSplit( @@ -276,7 +276,7 @@ public void testAssignEmptySplit() throws Exception { } @Test - public void testUsingCommittedOffsetsWithNoneOffsetResetStrategy() { + void testUsingCommittedOffsetsWithNoneOffsetResetStrategy() { final Properties props = new Properties(); props.setProperty( ConsumerConfig.GROUP_ID_CONFIG, "using-committed-offset-with-none-offset-reset"); @@ -321,7 +321,7 @@ public void testUsingCommittedOffsetsWithEarliestOrLatestOffsetResetStrategy( } @Test - public void testConsumerClientRackSupplier() { + void testConsumerClientRackSupplier() { String rackId = "use1-az1"; Properties properties = new Properties(); KafkaPartitionSplitReader reader = diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java index f5aa7f5fd..47aad9030 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/KafkaSourceReaderTest.java @@ -461,7 +461,7 @@ void testAssigningEmptySplitOnly() throws Exception { } @Test - public void testSupportsPausingOrResumingSplits() throws Exception { + void testSupportsPausingOrResumingSplits() throws Exception { final Set finishedSplits = new HashSet<>(); try (final KafkaSourceReader reader = @@ -507,7 +507,7 @@ public void testSupportsPausingOrResumingSplits() throws Exception { } @Test - public void testThatReaderDoesNotCallRackIdSupplierOnInit() throws Exception { + void testThatReaderDoesNotCallRackIdSupplierOnInit() throws Exception { SerializableSupplier rackIdSupplier = Mockito.mock(SerializableSupplier.class); try (KafkaSourceReader reader = @@ -525,7 +525,7 @@ public void testThatReaderDoesNotCallRackIdSupplierOnInit() throws Exception { } @Test - public void testThatReaderDoesCallRackIdSupplierOnSplitAssignment() throws Exception { + void testThatReaderDoesCallRackIdSupplierOnSplitAssignment() throws Exception { SerializableSupplier rackIdSupplier = Mockito.mock(SerializableSupplier.class); Mockito.when(rackIdSupplier.get()).thenReturn("use1-az1"); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java index b0ca63161..31ea90d7a 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java @@ -32,8 +32,8 @@ import org.apache.kafka.common.Configurable; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Collections; @@ -60,7 +60,7 @@ public void setUp() { } @Test - public void testKafkaDeserializationSchemaWrapper() throws Exception { + void testKafkaDeserializationSchemaWrapper() throws Exception { final ConsumerRecord consumerRecord = getConsumerRecord(); KafkaRecordDeserializationSchema schema = KafkaRecordDeserializationSchema.of(new JSONKeyValueDeserializationSchema(true)); @@ -79,7 +79,7 @@ public void testKafkaDeserializationSchemaWrapper() throws Exception { } @Test - public void testKafkaValueDeserializationSchemaWrapper() throws Exception { + void testKafkaValueDeserializationSchemaWrapper() throws Exception { final ConsumerRecord consumerRecord = getConsumerRecord(); KafkaRecordDeserializationSchema< org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node @@ -105,7 +105,7 @@ public void testKafkaValueDeserializationSchemaWrapper() throws Exception { } @Test - public void testKafkaValueDeserializerWrapper() throws Exception { + void testKafkaValueDeserializerWrapper() throws Exception { final String topic = "Topic"; byte[] value = new StringSerializer().serialize(topic, "world"); final ConsumerRecord consumerRecord = @@ -122,7 +122,7 @@ public void testKafkaValueDeserializerWrapper() throws Exception { } @Test - public void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Exception { + void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Exception { final Map config = Collections.singletonMap("simpleKey", "simpleValue"); KafkaRecordDeserializationSchema schema = KafkaRecordDeserializationSchema.valueOnly(SimpleStringSerializer.class, config); @@ -133,7 +133,7 @@ public void testKafkaValueDeserializerWrapperWithoutConfigurable() throws Except } @Test - public void testKafkaValueDeserializerWrapperWithConfigurable() throws Exception { + void testKafkaValueDeserializerWrapperWithConfigurable() throws Exception { final Map config = Collections.singletonMap("configKey", "configValue"); KafkaRecordDeserializationSchema schema = KafkaRecordDeserializationSchema.valueOnly( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java index db7647242..f73392310 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/split/KafkaPartitionSplitSerializerTest.java @@ -30,7 +30,7 @@ public class KafkaPartitionSplitSerializerTest { @Test - public void testSerializer() throws IOException { + void testSerializer() throws IOException { String topic = "topic"; Long offsetZero = 0L; Long normalOffset = 1L; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java index 8db2e596c..3693ec28d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkFixedPartitionerTest.java @@ -20,12 +20,12 @@ import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link FlinkFixedPartitioner}. */ -public class FlinkFixedPartitionerTest { +class FlinkFixedPartitionerTest { /** * Test for when there are more sinks than partitions. @@ -39,7 +39,7 @@ public class FlinkFixedPartitionerTest { * */ @Test - public void testMoreFlinkThanBrokers() { + void testMoreFlinkThanBrokers() { FlinkFixedPartitioner part = new FlinkFixedPartitioner<>(); int[] partitions = new int[] {0}; @@ -73,7 +73,7 @@ public void testMoreFlinkThanBrokers() { * */ @Test - public void testFewerPartitions() { + void testFewerPartitions() { FlinkFixedPartitioner part = new FlinkFixedPartitioner<>(); int[] partitions = new int[] {0, 1, 2, 3, 4}; @@ -93,7 +93,7 @@ public void testFewerPartitions() { * 3 ----------/ */ @Test - public void testMixedCase() { + void testMixedCase() { FlinkFixedPartitioner part = new FlinkFixedPartitioner<>(); int[] partitions = new int[] {0, 1}; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java index 296545cad..62032f014 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseMigrationTest.java @@ -36,12 +36,13 @@ import org.apache.flink.streaming.connectors.kafka.internals.KafkaTopicsDescriptor; import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; import org.apache.flink.streaming.util.OperatorSnapshotUtil; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.util.SerializedValue; -import org.junit.Ignore; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -66,7 +67,7 @@ *

For regenerating the binary snapshot files run {@link #writeSnapshot()} on the corresponding * Flink release-* branch. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class FlinkKafkaConsumerBaseMigrationTest { /** @@ -90,7 +91,7 @@ public class FlinkKafkaConsumerBaseMigrationTest { private final FlinkVersion testMigrateVersion; - @Parameterized.Parameters(name = "Migration Savepoint: {0}") + @Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_16); } @@ -100,9 +101,9 @@ public FlinkKafkaConsumerBaseMigrationTest(FlinkVersion testMigrateVersion) { } /** Manually run this to write binary snapshot data. */ - @Ignore + @Disabled @Test - public void writeSnapshot() throws Exception { + void writeSnapshot() throws Exception { writeSnapshot( "src/test/resources/kafka-consumer-migration-test-flink" + flinkGenerateSavepointVersion @@ -194,7 +195,7 @@ public void collect(String element) {} /** Test restoring from an legacy empty state, when no partitions could be found for topics. */ @Test - public void testRestoreFromEmptyStateNoPartitions() throws Exception { + void testRestoreFromEmptyStateNoPartitions() throws Exception { final DummyFlinkKafkaConsumer consumerFunction = new DummyFlinkKafkaConsumer<>( Collections.singletonList("dummy-topic"), @@ -235,7 +236,7 @@ public void testRestoreFromEmptyStateNoPartitions() throws Exception { * could be found for topics. */ @Test - public void testRestoreFromEmptyStateWithPartitions() throws Exception { + void testRestoreFromEmptyStateWithPartitions() throws Exception { final List partitions = new ArrayList<>(PARTITION_STATE.keySet()); final DummyFlinkKafkaConsumer consumerFunction = @@ -295,7 +296,7 @@ public void testRestoreFromEmptyStateWithPartitions() throws Exception { * partitions could be found for topics. */ @Test - public void testRestore() throws Exception { + void testRestore() throws Exception { final List partitions = new ArrayList<>(PARTITION_STATE.keySet()); final DummyFlinkKafkaConsumer consumerFunction = diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java index a2438165e..c9555f9de 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerBaseTest.java @@ -69,7 +69,7 @@ import org.apache.flink.util.function.SupplierWithException; import org.apache.flink.util.function.ThrowingRunnable; -import org.junit.Test; +import org.junit.jupiter.api.Test; import javax.annotation.Nonnull; @@ -102,14 +102,14 @@ import static org.mockito.Mockito.mock; /** Tests for the {@link FlinkKafkaConsumerBase}. */ -public class FlinkKafkaConsumerBaseTest extends TestLogger { +class FlinkKafkaConsumerBaseTest { private static final int maxParallelism = Short.MAX_VALUE / 2; /** Tests that not both types of timestamp extractors / watermark generators can be used. */ @Test @SuppressWarnings("unchecked") - public void testEitherWatermarkExtractor() { + void testEitherWatermarkExtractor() { assertThatThrownBy( () -> new DummyFlinkKafkaConsumer() @@ -142,7 +142,7 @@ public void testEitherWatermarkExtractor() { /** Tests that no checkpoints happen when the fetcher is not running. */ @Test - public void ignoreCheckpointWhenNotRunning() throws Exception { + void ignoreCheckpointWhenNotRunning() throws Exception { @SuppressWarnings("unchecked") final MockFetcher fetcher = new MockFetcher<>(); final FlinkKafkaConsumerBase consumer = @@ -170,7 +170,7 @@ public void ignoreCheckpointWhenNotRunning() throws Exception { * correctly contains the restored state instead. */ @Test - public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { + void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { @SuppressWarnings("unchecked") final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>(); @@ -203,7 +203,7 @@ public void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { } @Test - public void testConfigureOnCheckpointsCommitMode() throws Exception { + void testConfigureOnCheckpointsCommitMode() throws Exception { @SuppressWarnings("unchecked") // auto-commit enabled; this should be ignored in this case final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); @@ -216,7 +216,7 @@ public void testConfigureOnCheckpointsCommitMode() throws Exception { } @Test - public void testConfigureAutoCommitMode() throws Exception { + void testConfigureAutoCommitMode() throws Exception { @SuppressWarnings("unchecked") final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); @@ -226,7 +226,7 @@ public void testConfigureAutoCommitMode() throws Exception { } @Test - public void testConfigureDisableOffsetCommitWithCheckpointing() throws Exception { + void testConfigureDisableOffsetCommitWithCheckpointing() throws Exception { @SuppressWarnings("unchecked") // auto-commit enabled; this should be ignored in this case final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); @@ -241,7 +241,7 @@ public void testConfigureDisableOffsetCommitWithCheckpointing() throws Exception } @Test - public void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Exception { + void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Exception { @SuppressWarnings("unchecked") final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(false); @@ -255,7 +255,7 @@ public void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Except * (filterRestoredPartitionsWithDiscovered is active) */ @Test - public void testSetFilterRestoredParitionsNoChange() throws Exception { + void testSetFilterRestoredParitionsNoChange() throws Exception { checkFilterRestoredPartitionsWithDisovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -268,7 +268,7 @@ public void testSetFilterRestoredParitionsNoChange() throws Exception { * in restored partitions. (filterRestoredPartitionsWithDiscovered is active) */ @Test - public void testSetFilterRestoredParitionsWithRemovedTopic() throws Exception { + void testSetFilterRestoredParitionsWithRemovedTopic() throws Exception { checkFilterRestoredPartitionsWithDisovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1"}), @@ -281,7 +281,7 @@ public void testSetFilterRestoredParitionsWithRemovedTopic() throws Exception { * (filterRestoredPartitionsWithDiscovered is active) */ @Test - public void testSetFilterRestoredParitionsWithAddedTopic() throws Exception { + void testSetFilterRestoredParitionsWithAddedTopic() throws Exception { checkFilterRestoredPartitionsWithDisovered( Arrays.asList(new String[] {"kafka_topic_1"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -294,7 +294,7 @@ public void testSetFilterRestoredParitionsWithAddedTopic() throws Exception { * (filterRestoredPartitionsWithDiscovered is disabled) */ @Test - public void testDisableFilterRestoredParitionsNoChange() throws Exception { + void testDisableFilterRestoredParitionsNoChange() throws Exception { checkFilterRestoredPartitionsWithDisovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -307,7 +307,7 @@ public void testDisableFilterRestoredParitionsNoChange() throws Exception { * still in restored partitions. (filterRestoredPartitionsWithDiscovered is disabled) */ @Test - public void testDisableFilterRestoredParitionsWithRemovedTopic() throws Exception { + void testDisableFilterRestoredParitionsWithRemovedTopic() throws Exception { checkFilterRestoredPartitionsWithDisovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1"}), @@ -320,7 +320,7 @@ public void testDisableFilterRestoredParitionsWithRemovedTopic() throws Exceptio * (filterRestoredPartitionsWithDiscovered is disabled) */ @Test - public void testDisableFilterRestoredParitionsWithAddedTopic() throws Exception { + void testDisableFilterRestoredParitionsWithAddedTopic() throws Exception { checkFilterRestoredPartitionsWithDisovered( Arrays.asList(new String[] {"kafka_topic_1"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -598,7 +598,7 @@ public void go() throws Exception { } @Test - public void testClosePartitionDiscovererWhenOpenThrowException() throws Exception { + void testClosePartitionDiscovererWhenOpenThrowException() throws Exception { final RuntimeException failureCause = new RuntimeException(new FlinkException("Test partition discoverer exception")); final FailingPartitionDiscoverer failingPartitionDiscoverer = @@ -614,7 +614,7 @@ public void testClosePartitionDiscovererWhenOpenThrowException() throws Exceptio } @Test - public void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exception { + void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exception { final FlinkException failureCause = new FlinkException("Create Kafka fetcher failure."); final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer(); @@ -633,7 +633,7 @@ public void testClosePartitionDiscovererWhenCreateKafkaFetcherFails() throws Exc } @Test - public void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception { + void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception { final FlinkException failureCause = new FlinkException("Run Kafka fetcher failure."); // in this scenario, the partition discoverer will be concurrently accessed; @@ -674,7 +674,7 @@ private void testFailingConsumerLifecycle( } @Test - public void testClosePartitionDiscovererWithCancellation() throws Exception { + void testClosePartitionDiscovererWithCancellation() throws Exception { final DummyPartitionDiscoverer testPartitionDiscoverer = new DummyPartitionDiscoverer(); final TestingFlinkKafkaConsumer consumer = @@ -707,7 +707,7 @@ private void setupConsumer(FlinkKafkaConsumerBase consumer) throws Excep * that the two methods create compatible serializers. */ @Test - public void testExplicitStateSerializerCompatibility() throws Exception { + void testExplicitStateSerializerCompatibility() throws Exception { ExecutionConfig executionConfig = new ExecutionConfig(); Tuple2 tuple = @@ -733,12 +733,12 @@ public void testExplicitStateSerializerCompatibility() throws Exception { } @Test - public void testScaleUp() throws Exception { + void testScaleUp() throws Exception { testRescaling(5, 2, 8, 30); } @Test - public void testScaleDown() throws Exception { + void testScaleDown() throws Exception { testRescaling(5, 10, 2, 100); } @@ -883,7 +883,7 @@ private void testRescaling( } @Test - public void testOpen() throws Exception { + void testOpen() throws Exception { MockDeserializationSchema deserializationSchema = new MockDeserializationSchema<>(); AbstractStreamOperatorTestHarness testHarness = @@ -898,7 +898,7 @@ public void testOpen() throws Exception { } @Test - public void testOpenWithRestoreState() throws Exception { + void testOpenWithRestoreState() throws Exception { MockDeserializationSchema deserializationSchema = new MockDeserializationSchema<>(); final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java index 90c773730..74069be27 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java @@ -34,13 +34,13 @@ import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.junit.ClassRule; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.TestInstance.Lifecycle; import org.junit.jupiter.api.io.TempDir; +import org.testcontainers.junit.jupiter.Container; import java.nio.file.Path; import java.util.Properties; @@ -48,10 +48,10 @@ /** ITCase tests class for {@link FlinkKafkaConsumer}. */ @TestInstance(Lifecycle.PER_CLASS) -public class FlinkKafkaConsumerITCase { +class FlinkKafkaConsumerITCase { private static final String TOPIC1 = "FlinkKafkaConsumerITCase_topic1"; - @ClassRule + @Container public static final MiniClusterWithClientResource MINI_CLUSTER = new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() @@ -71,7 +71,7 @@ public void tearDown() throws Exception { } @Test - public void testStopWithSavepoint(@TempDir Path savepointsDir) throws Exception { + void testStopWithSavepoint(@TempDir Path savepointsDir) throws Exception { Configuration config = new Configuration() .set( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java index 15729a8c8..14045427f 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaInternalProducerITCase.java @@ -27,27 +27,30 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.time.Duration; import java.util.Collections; import java.util.Iterator; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.TimeUnit; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for our own {@link FlinkKafkaInternalProducer}. */ @SuppressWarnings("serial") -public class FlinkKafkaInternalProducerITCase extends KafkaTestBase { +class FlinkKafkaInternalProducerITCase extends KafkaTestBase { protected String transactionalId; protected Properties extraProperties; private volatile Exception exceptionInCallback; - @BeforeClass - public static void prepare() throws Exception { + @BeforeAll + protected static void prepare() throws Exception { LOG.info("-------------------------------------------------------------------------"); LOG.info(" Starting KafkaTestBase "); LOG.info("-------------------------------------------------------------------------"); @@ -63,8 +66,8 @@ public static void prepare() throws Exception { .setKafkaServerProperties(serverProperties)); } - @Before - public void before() { + @BeforeEach + void before() { transactionalId = UUID.randomUUID().toString(); extraProperties = new Properties(); extraProperties.putAll(standardProps); @@ -80,7 +83,8 @@ public void before() { extraProperties.put("isolation.level", "read_committed"); } - @Test(timeout = 60000L) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testHappyPath() throws Exception { String topicName = "flink-kafka-producer-happy-path"; @@ -101,7 +105,8 @@ public void testHappyPath() throws Exception { deleteTestTopic(topicName); } - @Test(timeout = 30000L) + @Test + @Timeout(30L) public void testResumeTransaction() throws Exception { String topicName = "flink-kafka-producer-resume-transaction"; FlinkKafkaInternalProducer kafkaProducer = @@ -147,61 +152,76 @@ public void testResumeTransaction() throws Exception { deleteTestTopic(topicName); } - @Test(timeout = 30000L, expected = IllegalStateException.class) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testPartitionsForAfterClosed() { FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); kafkaProducer.close(Duration.ofSeconds(5)); - kafkaProducer.partitionsFor("Topic"); + assertThatThrownBy(() -> kafkaProducer.partitionsFor("Topic")) + .isInstanceOf(IllegalStateException.class); } - @Test(timeout = 30000L, expected = IllegalStateException.class) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testInitTransactionsAfterClosed() { FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); kafkaProducer.close(Duration.ofSeconds(5)); - kafkaProducer.initTransactions(); + assertThatThrownBy(kafkaProducer::initTransactions) + .isInstanceOf(IllegalStateException.class); } - @Test(timeout = 30000L, expected = IllegalStateException.class) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testBeginTransactionAfterClosed() { FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); kafkaProducer.initTransactions(); kafkaProducer.close(Duration.ofSeconds(5)); - kafkaProducer.beginTransaction(); + assertThatThrownBy(kafkaProducer::beginTransaction) + .isInstanceOf(IllegalStateException.class); } - @Test(timeout = 30000L, expected = IllegalStateException.class) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testCommitTransactionAfterClosed() { String topicName = "testCommitTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); - kafkaProducer.commitTransaction(); + assertThatThrownBy(kafkaProducer::commitTransaction) + .isInstanceOf(IllegalStateException.class); } - @Test(timeout = 30000L, expected = IllegalStateException.class) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testResumeTransactionAfterClosed() { String topicName = "testAbortTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); - kafkaProducer.resumeTransaction(0L, (short) 1); + assertThatThrownBy(() -> kafkaProducer.resumeTransaction(0L, (short) 1)) + .isInstanceOf(IllegalStateException.class); } - @Test(timeout = 30000L, expected = IllegalStateException.class) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testAbortTransactionAfterClosed() { String topicName = "testAbortTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); kafkaProducer.abortTransaction(); - kafkaProducer.resumeTransaction(0L, (short) 1); + assertThatThrownBy(() -> kafkaProducer.resumeTransaction(0L, (short) 1)) + .isInstanceOf(IllegalStateException.class); } - @Test(timeout = 30000L, expected = IllegalStateException.class) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testFlushAfterClosed() { String topicName = "testCommitTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); - kafkaProducer.flush(); + assertThatThrownBy(kafkaProducer::flush) + .isInstanceOf(IllegalStateException.class); } - @Test(timeout = 30000L) + @Test + @Timeout(value = 30, unit = TimeUnit.SECONDS) public void testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator() throws Exception { String topic = "flink-kafka-producer-txn-coordinator-changed-" + UUID.randomUUID(); createTestTopic(topic, 1, 1); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java index 4274fcff2..5af52ec54 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerBaseTest.java @@ -40,15 +40,18 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.junit.Test; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import java.util.ArrayList; import java.util.List; import java.util.Properties; +import java.util.concurrent.TimeUnit; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyString; @@ -58,16 +61,22 @@ import static org.mockito.Mockito.when; /** Tests for the {@link FlinkKafkaProducerBase}. */ -public class FlinkKafkaProducerBaseTest { +class FlinkKafkaProducerBaseTest { /** Tests that the constructor eagerly checks bootstrap servers are set in config. */ - @Test(expected = IllegalArgumentException.class) + @Test public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception { // no bootstrap servers set in props Properties props = new Properties(); // should throw IllegalArgumentException - new DummyFlinkKafkaProducer<>( - props, new KeyedSerializationSchemaWrapper<>(new SimpleStringSchema()), null); + assertThatThrownBy( + () -> + new DummyFlinkKafkaProducer<>( + props, + new KeyedSerializationSchemaWrapper<>( + new SimpleStringSchema()), + null)) + .isInstanceOf(IllegalArgumentException.class); } /** @@ -75,7 +84,7 @@ public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception * deserializers if not set. */ @Test - public void testKeyValueDeserializersSetIfMissing() throws Exception { + void testKeyValueDeserializersSetIfMissing() throws Exception { Properties props = new Properties(); props.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:12345"); // should set missing key value deserializers @@ -94,7 +103,7 @@ public void testKeyValueDeserializersSetIfMissing() throws Exception { /** Tests that partitions list is determinate and correctly provided to custom partitioner. */ @SuppressWarnings("unchecked") @Test - public void testPartitionerInvokedWithDeterminatePartitionList() throws Exception { + void testPartitionerInvokedWithDeterminatePartitionList() throws Exception { FlinkKafkaPartitioner mockPartitioner = mock(FlinkKafkaPartitioner.class); RuntimeContext mockRuntimeContext = mock(StreamingRuntimeContext.class); @@ -141,7 +150,7 @@ public void testPartitionerInvokedWithDeterminatePartitionList() throws Exceptio * should be rethrown. */ @Test - public void testAsyncErrorRethrownOnInvoke() throws Throwable { + void testAsyncErrorRethrownOnInvoke() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), @@ -178,7 +187,7 @@ public void testAsyncErrorRethrownOnInvoke() throws Throwable { * should be rethrown. */ @Test - public void testAsyncErrorRethrownOnCheckpoint() throws Throwable { + void testAsyncErrorRethrownOnCheckpoint() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), @@ -219,8 +228,9 @@ public void testAsyncErrorRethrownOnCheckpoint() throws Throwable { * pending records. The test for that is covered in testAtLeastOnceProducer. */ @SuppressWarnings("unchecked") - @Test(timeout = 5000) - public void testAsyncErrorRethrownOnCheckpointAfterFlush() throws Throwable { + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + void testAsyncErrorRethrownOnCheckpointAfterFlush() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), @@ -280,8 +290,9 @@ public void go() throws Exception { * the test will not finish if the logic is broken. */ @SuppressWarnings("unchecked") - @Test(timeout = 10000) - public void testAtLeastOnceProducer() throws Throwable { + @Test + @Timeout(value = 10, unit = TimeUnit.SECONDS) + void testAtLeastOnceProducer() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), @@ -353,8 +364,9 @@ public void go() throws Exception { * records; we set a timeout because the test will not finish if the logic is broken. */ @SuppressWarnings("unchecked") - @Test(timeout = 5000) - public void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwable { + @Test + @Timeout(value = 5, unit = TimeUnit.SECONDS) + void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( FakeStandardProducerConfig.get(), diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java index 7b345bf7d..50802b896 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerITCase.java @@ -30,9 +30,9 @@ import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; import org.apache.kafka.common.errors.ProducerFencedException; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.Arrays; @@ -57,7 +57,7 @@ *

Do not run this class in the same junit execution with other tests in your IDE. This may lead * leaking threads. */ -public class FlinkKafkaProducerITCase extends KafkaTestBase { +class FlinkKafkaProducerITCase extends KafkaTestBase { protected String transactionalId; protected Properties extraProperties; @@ -68,8 +68,8 @@ public class FlinkKafkaProducerITCase extends KafkaTestBase { protected KeyedSerializationSchema integerKeyedSerializationSchema = new KeyedSerializationSchemaWrapper<>(integerSerializationSchema); - @Before - public void before() { + @BeforeEach + void before() { transactionalId = UUID.randomUUID().toString(); extraProperties = new Properties(); extraProperties.putAll(standardProps); @@ -86,12 +86,12 @@ public void before() { } @Test - public void resourceCleanUpNone() throws Exception { + void resourceCleanUpNone() throws Exception { resourceCleanUp(FlinkKafkaProducer.Semantic.NONE); } @Test - public void resourceCleanUpAtLeastOnce() throws Exception { + void resourceCleanUpAtLeastOnce() throws Exception { resourceCleanUp(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); } @@ -125,7 +125,7 @@ public void resourceCleanUp(FlinkKafkaProducer.Semantic semantic) throws Excepti * will not clash with previous transactions using same transactional.ids. */ @Test - public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exception { + void testRestoreToCheckpointAfterExceedingProducersPool() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; try (OneInputStreamOperatorTestHarness testHarness1 = @@ -171,8 +171,8 @@ public void testRestoreToCheckpointAfterExceedingProducersPool() throws Exceptio /** This test hangs when running it in your IDE. */ @Test - @Ignore - public void testFlinkKafkaProducerFailBeforeNotify() throws Exception { + @Disabled + void testFlinkKafkaProducerFailBeforeNotify() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; final OneInputStreamOperatorTestHarness testHarness = @@ -220,7 +220,7 @@ public void testFlinkKafkaProducerFailBeforeNotify() throws Exception { * committed records that were created after this lingering transaction. */ @Test - public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { + void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { String topic = "flink-kafka-producer-fail-before-notify"; OneInputStreamOperatorTestHarness testHarness1 = createTestHarness(topic); @@ -274,7 +274,7 @@ public void testFailBeforeNotifyAndResumeWorkAfterwards() throws Exception { } @Test - public void testFailAndRecoverSameCheckpointTwice() throws Exception { + void testFailAndRecoverSameCheckpointTwice() throws Exception { String topic = "flink-kafka-producer-fail-and-recover-same-checkpoint-twice"; OperatorSubtaskState snapshot1; @@ -331,7 +331,7 @@ public void testFailAndRecoverSameCheckpointTwice() throws Exception { * read committed records that were created after this lingering transaction. */ @Test - public void testScaleDownBeforeFirstCheckpoint() throws Exception { + void testScaleDownBeforeFirstCheckpoint() throws Exception { String topic = "scale-down-before-first-checkpoint"; List operatorsToClose = new ArrayList<>(); @@ -401,7 +401,7 @@ public void testScaleDownBeforeFirstCheckpoint() throws Exception { * so it has to generate new ones that are greater then 4. */ @Test - public void testScaleUpAfterScalingDown() throws Exception { + void testScaleUpAfterScalingDown() throws Exception { String topic = "scale-up-after-scaling-down"; final int parallelism1 = 4; @@ -531,7 +531,7 @@ private OperatorSubtaskState repartitionAndExecute( } @Test - public void testRecoverCommittedTransaction() throws Exception { + void testRecoverCommittedTransaction() throws Exception { String topic = "flink-kafka-producer-recover-committed-transaction"; OneInputStreamOperatorTestHarness testHarness = createTestHarness(topic); @@ -560,7 +560,7 @@ public void testRecoverCommittedTransaction() throws Exception { } @Test - public void testRunOutOfProducersInThePool() throws Exception { + void testRunOutOfProducersInThePool() throws Exception { String topic = "flink-kafka-run-out-of-producers"; try (OneInputStreamOperatorTestHarness testHarness = @@ -583,7 +583,7 @@ public void testRunOutOfProducersInThePool() throws Exception { } @Test - public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { + void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { String topic = "testMigrateFromAtLeastOnceToExactlyOnce"; testRecoverWithChangeSemantics( topic, @@ -594,7 +594,7 @@ public void testMigrateFromAtLeastOnceToExactlyOnce() throws Exception { } @Test - public void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception { + void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception { String topic = "testMigrateFromExactlyOnceToAtLeastOnce"; testRecoverWithChangeSemantics( topic, @@ -605,7 +605,7 @@ public void testMigrateFromAtExactlyOnceToAtLeastOnce() throws Exception { } @Test - public void testDefaultTransactionalIdPrefix() throws Exception { + void testDefaultTransactionalIdPrefix() throws Exception { Properties properties = createProperties(); String topic = "testCustomizeTransactionalIdPrefix"; FlinkKafkaProducer kafkaProducer = @@ -642,7 +642,7 @@ public void testDefaultTransactionalIdPrefix() throws Exception { } @Test - public void testCustomizeTransactionalIdPrefix() throws Exception { + void testCustomizeTransactionalIdPrefix() throws Exception { String transactionalIdPrefix = "my-prefix"; Properties properties = createProperties(); @@ -675,7 +675,7 @@ public void testCustomizeTransactionalIdPrefix() throws Exception { } @Test - public void testRestoreUsingDifferentTransactionalIdPrefix() throws Exception { + void testRestoreUsingDifferentTransactionalIdPrefix() throws Exception { String topic = "testCustomizeTransactionalIdPrefix"; Properties properties = createProperties(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java index 5e87f04b8..3041d2e7c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationOperatorTest.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.FlinkVersion; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.junit.Ignore; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Disabled; import java.util.Arrays; import java.util.Collection; @@ -34,8 +34,8 @@ * by the {@link #getOperatorSnapshotPath(FlinkVersion)} method then copy the resource to the path * also specified by the {@link #getOperatorSnapshotPath(FlinkVersion)} method. */ -public class FlinkKafkaProducerMigrationOperatorTest extends FlinkKafkaProducerMigrationTest { - @Parameterized.Parameters(name = "Migration Savepoint: {0}") +class FlinkKafkaProducerMigrationOperatorTest extends FlinkKafkaProducerMigrationTest { + @Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { return Arrays.asList( FlinkVersion.v1_8, FlinkVersion.v1_9, FlinkVersion.v1_10, FlinkVersion.v1_11); @@ -52,7 +52,7 @@ public String getOperatorSnapshotPath(FlinkVersion version) { + "-snapshot"; } - @Ignore + @Disabled @Override public void writeSnapshot() throws Exception { throw new UnsupportedOperationException(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java index 98ab88296..9ea613e89 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerMigrationTest.java @@ -23,10 +23,11 @@ import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.streaming.api.operators.StreamSink; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.kafka.clients.producer.ProducerConfig; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Collection; import java.util.Properties; @@ -38,9 +39,9 @@ *

For regenerating the binary snapshot files run {@link #writeSnapshot()} on the corresponding * Flink release-* branch. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class FlinkKafkaProducerMigrationTest extends KafkaMigrationTestBase { - @Parameterized.Parameters(name = "Migration Savepoint: {0}") + @Parameters(name = "Migration Savepoint: {0}") public static Collection parameters() { return FlinkVersion.rangeOf(FlinkVersion.v1_8, FlinkVersion.v1_16); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java index 6fedcc43c..4fb380820 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducerTest.java @@ -26,7 +26,7 @@ import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.kafka.clients.producer.ProducerRecord; -import org.junit.Test; +import org.junit.jupiter.api.Test; import javax.annotation.Nullable; @@ -34,11 +34,12 @@ import java.util.Properties; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Tests for {@link FlinkKafkaProducer}. */ -public class FlinkKafkaProducerTest { +class FlinkKafkaProducerTest { @Test - public void testOpenSerializationSchemaProducer() throws Exception { + void testOpenSerializationSchemaProducer() throws Exception { OpenTestingSerializationSchema schema = new OpenTestingSerializationSchema(); FlinkKafkaProducer kafkaProducer = new FlinkKafkaProducer<>("localhost:9092", "test-topic", schema); @@ -58,7 +59,7 @@ public void testOpenSerializationSchemaProducer() throws Exception { } @Test - public void testOpenKafkaSerializationSchemaProducer() throws Exception { + void testOpenKafkaSerializationSchemaProducer() throws Exception { OpenTestingKafkaSerializationSchema schema = new OpenTestingKafkaSerializationSchema(); Properties properties = new Properties(); properties.put("bootstrap.servers", "localhost:9092"); @@ -84,7 +85,7 @@ public void testOpenKafkaSerializationSchemaProducer() throws Exception { } @Test - public void testOpenKafkaCustomPartitioner() throws Exception { + void testOpenKafkaCustomPartitioner() throws Exception { CustomPartitioner partitioner = new CustomPartitioner<>(); Properties properties = new Properties(); properties.put("bootstrap.servers", "localhost:9092"); @@ -109,12 +110,13 @@ public void testOpenKafkaCustomPartitioner() throws Exception { assertThat(partitioner.openCalled).isTrue(); } - @Test(expected = NullPointerException.class) + @Test public void testProvidedNullTransactionalIdPrefix() { FlinkKafkaProducer kafkaProducer = new FlinkKafkaProducer<>( "localhost:9092", "test-topic", new OpenTestingSerializationSchema()); - kafkaProducer.setTransactionalIdPrefix(null); + assertThatThrownBy(() -> kafkaProducer.setTransactionalIdPrefix(null)) + .isInstanceOf(NullPointerException.class); } private static class CustomPartitioner extends FlinkKafkaPartitioner { diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java index a5abb5e6d..e42e7dd0b 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/JSONKeyValueDeserializationSchemaTest.java @@ -24,17 +24,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.junit.Test; +import org.junit.jupiter.api.Test; import static org.assertj.core.api.Assertions.assertThat; /** Tests for the{@link JSONKeyValueDeserializationSchema}. */ -public class JSONKeyValueDeserializationSchemaTest { +class JSONKeyValueDeserializationSchemaTest { private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); @Test - public void testDeserializeWithoutMetadata() throws Exception { + void testDeserializeWithoutMetadata() throws Exception { ObjectNode initialKey = OBJECT_MAPPER.createObjectNode(); initialKey.put("index", 4); byte[] serializedKey = OBJECT_MAPPER.writeValueAsBytes(initialKey); @@ -54,7 +54,7 @@ public void testDeserializeWithoutMetadata() throws Exception { } @Test - public void testDeserializeWithoutKey() throws Exception { + void testDeserializeWithoutKey() throws Exception { byte[] serializedKey = null; ObjectNode initialValue = OBJECT_MAPPER.createObjectNode(); @@ -87,7 +87,7 @@ private static ConsumerRecord newConsumerRecord( } @Test - public void testDeserializeWithoutValue() throws Exception { + void testDeserializeWithoutValue() throws Exception { ObjectNode initialKey = OBJECT_MAPPER.createObjectNode(); initialKey.put("index", 4); byte[] serializedKey = OBJECT_MAPPER.writeValueAsBytes(initialKey); @@ -105,7 +105,7 @@ public void testDeserializeWithoutValue() throws Exception { } @Test - public void testDeserializeWithMetadata() throws Exception { + void testDeserializeWithMetadata() throws Exception { ObjectNode initialKey = OBJECT_MAPPER.createObjectNode(); initialKey.put("index", 4); byte[] serializedKey = OBJECT_MAPPER.writeValueAsBytes(initialKey); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java index 88f6ac607..a38d5e975 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaConsumerTestBase.java @@ -88,7 +88,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.NotLeaderForPartitionException; import org.apache.kafka.common.errors.TimeoutException; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import javax.annotation.Nullable; import javax.management.MBeanServer; @@ -143,8 +143,8 @@ protected KafkaConsumerTestBase(boolean useNewSource) { * Makes sure that no job is on the JobManager any more from any previous tests that use the * same mini cluster. Otherwise, missing slots may happen. */ - @Before - public void setClientAndEnsureNoJobIsLingering() throws Exception { + @BeforeEach + void setClientAndEnsureNoJobIsLingering() throws Exception { client = flink.getClusterClient(); waitUntilNoJobIsRunning(client); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java index 68db69187..9669df535 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaITCase.java @@ -38,8 +38,9 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import javax.annotation.Nullable; @@ -47,12 +48,13 @@ import java.io.IOException; import java.util.Optional; import java.util.UUID; +import java.util.concurrent.TimeUnit; /** IT cases for Kafka. */ -public class KafkaITCase extends KafkaConsumerTestBase { +class KafkaITCase extends KafkaConsumerTestBase { - @BeforeClass - public static void prepare() throws Exception { + @BeforeAll + protected static void prepare() throws Exception { KafkaProducerTestBase.prepare(); ((KafkaTestEnvironmentImpl) kafkaServer) .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); @@ -62,130 +64,153 @@ public static void prepare() throws Exception { // Suite of Tests // ------------------------------------------------------------------------ - @Test(timeout = 120000) + @Test + @Timeout(value = 120, unit = TimeUnit.SECONDS) public void testFailOnNoBroker() throws Exception { runFailOnNoBrokerTest(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testConcurrentProducerConsumerTopology() throws Exception { runSimpleConcurrentProducerConsumerTopology(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testKeyValueSupport() throws Exception { runKeyValueTest(); } // --- canceling / failures --- - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testCancelingEmptyTopic() throws Exception { runCancelingOnEmptyInputTest(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testCancelingFullTopic() throws Exception { runCancelingOnFullInputTest(); } // --- source to partition mappings and exactly once --- - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testOneToOneSources() throws Exception { runOneToOneExactlyOnceTest(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testOneSourceMultiplePartitions() throws Exception { runOneSourceMultiplePartitionsExactlyOnceTest(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testMultipleSourcesOnePartition() throws Exception { runMultipleSourcesOnePartitionExactlyOnceTest(); } // --- broker failure --- - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testBrokerFailure() throws Exception { runBrokerFailureTest(); } // --- special executions --- - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testBigRecordJob() throws Exception { runBigRecordTestTopology(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testMultipleTopicsWithLegacySerializer() throws Exception { runProduceConsumeMultipleTopics(true); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testMultipleTopicsWithKafkaSerializer() throws Exception { runProduceConsumeMultipleTopics(false); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testAllDeletes() throws Exception { runAllDeletesTest(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testMetricsAndEndOfStream() throws Exception { runEndOfStreamTest(); } // --- startup mode --- - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testStartFromEarliestOffsets() throws Exception { runStartFromEarliestOffsets(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testStartFromLatestOffsets() throws Exception { runStartFromLatestOffsets(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testStartFromGroupOffsets() throws Exception { runStartFromGroupOffsets(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testStartFromSpecificOffsets() throws Exception { runStartFromSpecificOffsets(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testStartFromTimestamp() throws Exception { runStartFromTimestamp(); } // --- offset committing --- - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testCommitOffsetsToKafka() throws Exception { runCommitOffsetsToKafka(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception { runAutoOffsetRetrievalAndCommitToKafka(); } - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testCollectingSchema() throws Exception { runCollectingSchemaTest(); } /** Kafka 20 specific test, ensuring Timestamps are properly written to and read from Kafka. */ - @Test(timeout = 60000) + @Test + @Timeout(value = 60, unit = TimeUnit.SECONDS) public void testTimestamps() throws Exception { final String topic = "tstopic-" + UUID.randomUUID(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaMigrationTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaMigrationTestBase.java index b08bb05e3..60cc247ae 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaMigrationTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaMigrationTestBase.java @@ -27,10 +27,10 @@ import org.apache.flink.streaming.util.OperatorSnapshotUtil; import org.apache.flink.streaming.util.serialization.KeyedSerializationSchema; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Ignore; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Disabled; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -79,20 +79,20 @@ public String getOperatorSnapshotPath(FlinkVersion version) { * Override {@link KafkaTestBase}. Kafka Migration Tests are starting up Kafka/ZooKeeper cluster * manually */ - @BeforeClass - public static void prepare() throws Exception {} + @BeforeAll + protected static void prepare() throws Exception {} /** * Override {@link KafkaTestBase}. Kafka Migration Tests are starting up Kafka/ZooKeeper cluster * manually */ - @AfterClass - public static void shutDownServices() throws Exception {} + @AfterAll + protected static void shutDownServices() throws Exception {} /** Manually run this to write binary snapshot data. */ - @Ignore + @Disabled @Test - public void writeSnapshot() throws Exception { + void writeSnapshot() throws Exception { try { checkState(flinkGenerateSavepointVersion.isPresent()); startClusters(); @@ -129,7 +129,7 @@ private OperatorSubtaskState initializeTestState() throws Exception { @SuppressWarnings("warning") @Test - public void testRestoreProducer() throws Exception { + void testRestoreProducer() throws Exception { try { startClusters(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerAtLeastOnceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerAtLeastOnceITCase.java index aae2680bd..405a371c8 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerAtLeastOnceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerAtLeastOnceITCase.java @@ -18,14 +18,14 @@ package org.apache.flink.streaming.connectors.kafka; -import org.junit.BeforeClass; +import org.junit.jupiter.api.BeforeAll; /** IT cases for the {@link FlinkKafkaProducer}. */ @SuppressWarnings("serial") -public class KafkaProducerAtLeastOnceITCase extends KafkaProducerTestBase { +class KafkaProducerAtLeastOnceITCase extends KafkaProducerTestBase { - @BeforeClass - public static void prepare() throws Exception { + @BeforeAll + protected static void prepare() throws Exception { KafkaProducerTestBase.prepare(); ((KafkaTestEnvironmentImpl) kafkaServer) .setProducerSemantic(FlinkKafkaProducer.Semantic.AT_LEAST_ONCE); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExactlyOnceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExactlyOnceITCase.java index f8b20ae02..2a1525b60 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExactlyOnceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerExactlyOnceITCase.java @@ -18,21 +18,21 @@ package org.apache.flink.streaming.connectors.kafka; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; /** IT cases for the {@link FlinkKafkaProducer}. */ @SuppressWarnings("serial") -public class KafkaProducerExactlyOnceITCase extends KafkaProducerTestBase { - @BeforeClass - public static void prepare() throws Exception { +class KafkaProducerExactlyOnceITCase extends KafkaProducerTestBase { + @BeforeAll + protected static void prepare() throws Exception { KafkaProducerTestBase.prepare(); ((KafkaTestEnvironmentImpl) kafkaServer) .setProducerSemantic(FlinkKafkaProducer.Semantic.EXACTLY_ONCE); } @Test - public void testMultipleSinkOperators() throws Exception { + void testMultipleSinkOperators() throws Exception { testExactlyOnce(false, 2); } } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java index 624381068..bd09fd8db 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaProducerTestBase.java @@ -40,7 +40,7 @@ import org.apache.flink.test.util.TestUtils; import org.apache.flink.util.Preconditions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.io.Serializable; import java.util.ArrayList; @@ -87,7 +87,7 @@ public abstract class KafkaProducerTestBase extends KafkaTestBaseWithFlink { * partitions are present. */ @Test - public void testCustomPartitioning() { + void testCustomPartitioning() { try { LOG.info("Starting KafkaProducerITCase.testCustomPartitioning()"); @@ -204,13 +204,13 @@ public void cancel() { /** Tests the exactly-once semantic for the simple writes into Kafka. */ @Test - public void testExactlyOnceRegularSink() throws Exception { + void testExactlyOnceRegularSink() throws Exception { testExactlyOnce(true, 1); } /** Tests the exactly-once semantic for the simple writes into Kafka. */ @Test - public void testExactlyOnceCustomOperator() throws Exception { + void testExactlyOnceCustomOperator() throws Exception { testExactlyOnce(false, 1); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index 9fb16d40c..17ac62602 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -34,14 +34,15 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.testcontainers.junit.jupiter.Container; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Serializable; +import java.nio.file.Path; import java.util.Properties; import java.util.UUID; @@ -65,7 +66,7 @@ public class KafkaShortRetentionTestBase implements Serializable { private static KafkaTestEnvironment kafkaServer; private static Properties standardProps; - @ClassRule + @Container public static MiniClusterWithClientResource flink = new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() @@ -74,7 +75,7 @@ public class KafkaShortRetentionTestBase implements Serializable { .setNumberSlotsPerTaskManager(TM_SLOTS) .build()); - @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir public Path tempFolder; protected static Properties secureProps = new Properties(); @@ -84,8 +85,8 @@ private static Configuration getConfiguration() { return flinkConfig; } - @BeforeClass - public static void prepare() throws Exception { + @BeforeAll + static void prepare() throws Exception { LOG.info("-------------------------------------------------------------------------"); LOG.info(" Starting KafkaShortRetentionTestBase "); LOG.info("-------------------------------------------------------------------------"); @@ -113,8 +114,8 @@ public static void prepare() throws Exception { standardProps = kafkaServer.getStandardProperties(); } - @AfterClass - public static void shutDownServices() throws Exception { + @AfterAll + static void shutDownServices() throws Exception { kafkaServer.shutdown(); secureProps.clear(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index b4fd54ab2..3b0e0ae79 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -28,7 +28,6 @@ import org.apache.flink.streaming.util.TestStreamEnvironment; import org.apache.flink.test.util.SuccessException; import org.apache.flink.testutils.junit.RetryOnFailure; -import org.apache.flink.testutils.junit.RetryRule; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.TestLogger; @@ -37,11 +36,11 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -92,18 +91,16 @@ public abstract class KafkaTestBase extends TestLogger { public static KafkaTestEnvironment kafkaServer; - @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir public static File temporaryFolder; public static Properties secureProps = new Properties(); - @Rule public final RetryRule retryRule = new RetryRule(); - // ------------------------------------------------------------------------ // Setup and teardown of the mini clusters // ------------------------------------------------------------------------ - @BeforeClass - public static void prepare() throws Exception { + @BeforeAll + protected static void prepare() throws Exception { LOG.info("-------------------------------------------------------------------------"); LOG.info(" Starting KafkaTestBase "); LOG.info("-------------------------------------------------------------------------"); @@ -111,8 +108,8 @@ public static void prepare() throws Exception { startClusters(false); } - @AfterClass - public static void shutDownServices() throws Exception { + @AfterAll + protected static void shutDownServices() throws Exception { LOG.info("-------------------------------------------------------------------------"); LOG.info(" Shut down KafkaTestBase "); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java index 1c8d651f0..163b25cb2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java @@ -20,7 +20,7 @@ import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.junit.ClassRule; +import org.testcontainers.junit.jupiter.Container; /** The base for the Kafka tests with Flink's MiniCluster. */ @SuppressWarnings("serial") @@ -30,7 +30,7 @@ public abstract class KafkaTestBaseWithFlink extends KafkaTestBase { protected static final int TM_SLOTS = 8; - @ClassRule + @Container public static MiniClusterWithClientResource flink = new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java index 92fc08eb7..7572c71d2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherTest.java @@ -28,7 +28,7 @@ import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; import org.apache.flink.util.SerializedValue; -import org.junit.Test; +import org.junit.jupiter.api.Test; import javax.annotation.Nonnull; @@ -43,10 +43,10 @@ /** Tests for the {@link AbstractFetcher}. */ @SuppressWarnings("serial") -public class AbstractFetcherTest { +class AbstractFetcherTest { @Test - public void testIgnorePartitionStateSentinelInSnapshot() throws Exception { + void testIgnorePartitionStateSentinelInSnapshot() throws Exception { final String testTopic = "test topic name"; Map originalPartitions = new HashMap<>(); originalPartitions.put( @@ -93,7 +93,7 @@ public void onException(Throwable cause) { // ------------------------------------------------------------------------ @Test - public void testSkipCorruptedRecord() throws Exception { + void testSkipCorruptedRecord() throws Exception { final String testTopic = "test topic name"; Map originalPartitions = new HashMap<>(); originalPartitions.put( @@ -127,7 +127,7 @@ public void testSkipCorruptedRecord() throws Exception { } @Test - public void testConcurrentPartitionsDiscoveryAndLoopFetching() throws Exception { + void testConcurrentPartitionsDiscoveryAndLoopFetching() throws Exception { // test data final KafkaTopicPartition testPartition = new KafkaTopicPartition("test", 42); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java index 6cd3b6453..c393bd306 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java @@ -31,12 +31,15 @@ import org.apache.flink.streaming.runtime.operators.util.AssignerWithPunctuatedWatermarksAdapter; import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.util.SerializedValue; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.experimental.runners.Enclosed; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -54,14 +57,13 @@ /** Tests for the watermarking behaviour of {@link AbstractFetcher}. */ @SuppressWarnings("serial") -@RunWith(Enclosed.class) -public class AbstractFetcherWatermarksTest { +class AbstractFetcherWatermarksTest { /** Tests with watermark generators that have a periodic nature. */ - @RunWith(Parameterized.class) + @ExtendWith(ParameterizedTestExtension.class) public static class PeriodicWatermarksSuite { - @Parameterized.Parameters + @Parameters public static Collection> getParams() { return Arrays.asList( new AssignerWithPeriodicWatermarksAdapter.Strategy<>( @@ -70,10 +72,10 @@ public static Collection> getParams() { .withTimestampAssigner((event, previousTimestamp) -> event)); } - @Parameterized.Parameter public WatermarkStrategy testWmStrategy; + @Parameter public WatermarkStrategy testWmStrategy; @Test - public void testPeriodicWatermarks() throws Exception { + void testPeriodicWatermarks() throws Exception { final String testTopic = "test topic name"; Map originalPartitions = new HashMap<>(); originalPartitions.put( @@ -161,7 +163,7 @@ public void testPeriodicWatermarks() throws Exception { } @Test - public void testSkipCorruptedRecordWithPeriodicWatermarks() throws Exception { + void testSkipCorruptedRecordWithPeriodicWatermarks() throws Exception { final String testTopic = "test topic name"; Map originalPartitions = new HashMap<>(); originalPartitions.put( @@ -212,7 +214,7 @@ public void testSkipCorruptedRecordWithPeriodicWatermarks() throws Exception { } @Test - public void testPeriodicWatermarksWithNoSubscribedPartitionsShouldYieldNoWatermarks() + void testPeriodicWatermarksWithNoSubscribedPartitionsShouldYieldNoWatermarks() throws Exception { final String testTopic = "test topic name"; Map originalPartitions = new HashMap<>(); @@ -248,7 +250,7 @@ public void testPeriodicWatermarksWithNoSubscribedPartitionsShouldYieldNoWaterma public static class PunctuatedWatermarksSuite { @Test - public void testSkipCorruptedRecordWithPunctuatedWatermarks() throws Exception { + void testSkipCorruptedRecordWithPunctuatedWatermarks() throws Exception { final String testTopic = "test topic name"; Map originalPartitions = new HashMap<>(); originalPartitions.put( @@ -296,7 +298,7 @@ public void testSkipCorruptedRecordWithPunctuatedWatermarks() throws Exception { } @Test - public void testPunctuatedWatermarks() throws Exception { + void testPunctuatedWatermarks() throws Exception { final String testTopic = "test topic name"; Map originalPartitions = new HashMap<>(); originalPartitions.put( diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java index b47534596..349941499 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java @@ -20,7 +20,10 @@ import org.apache.flink.streaming.connectors.kafka.testutils.TestPartitionDiscoverer; -import org.junit.Test; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -40,7 +43,7 @@ * Tests that the partition assignment in the partition discoverer is deterministic and stable, with * both fixed and growing partitions. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class AbstractPartitionDiscovererTest { private static final String TEST_TOPIC = "test-topic"; @@ -52,7 +55,7 @@ public AbstractPartitionDiscovererTest(KafkaTopicsDescriptor topicsDescriptor) { this.topicsDescriptor = topicsDescriptor; } - @Parameterized.Parameters(name = "KafkaTopicsDescriptor = {0}") + @Parameters(name = "KafkaTopicsDescriptor = {0}") @SuppressWarnings("unchecked") public static Collection timeCharacteristic() { return Arrays.asList( @@ -65,7 +68,7 @@ public static Collection timeCharacteristic() { } @Test - public void testPartitionsEqualConsumersFixedPartitions() throws Exception { + void testPartitionsEqualConsumersFixedPartitions() throws Exception { List mockGetAllPartitionsForTopicsReturn = Arrays.asList( new KafkaTopicPartition(TEST_TOPIC, 0), @@ -117,7 +120,7 @@ public void testPartitionsEqualConsumersFixedPartitions() throws Exception { } @Test - public void testMultiplePartitionsPerConsumersFixedPartitions() { + void testMultiplePartitionsPerConsumersFixedPartitions() { try { final int[] partitionIDs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; @@ -186,7 +189,7 @@ public void testMultiplePartitionsPerConsumersFixedPartitions() { } @Test - public void testPartitionsFewerThanConsumersFixedPartitions() { + void testPartitionsFewerThanConsumersFixedPartitions() { try { List mockGetAllPartitionsForTopicsReturn = Arrays.asList( @@ -248,7 +251,7 @@ public void testPartitionsFewerThanConsumersFixedPartitions() { } @Test - public void testGrowingPartitions() { + void testGrowingPartitions() { try { final int[] newPartitionIDs = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; List allPartitions = new ArrayList<>(11); @@ -417,7 +420,7 @@ public void testGrowingPartitions() { } @Test - public void testDeterministicAssignmentWithDifferentFetchedPartitionOrdering() + void testDeterministicAssignmentWithDifferentFetchedPartitionOrdering() throws Exception { int numSubtasks = 4; @@ -478,7 +481,7 @@ public void testDeterministicAssignmentWithDifferentFetchedPartitionOrdering() } @Test - public void testNonContiguousPartitionIdDiscovery() throws Exception { + void testNonContiguousPartitionIdDiscovery() throws Exception { List mockGetAllPartitionsForTopicsReturn1 = Arrays.asList( new KafkaTopicPartition("test-topic", 1), diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java index 8697b1486..cd9e9d38d 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/ClosableBlockingQueueTest.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.ArrayList; import java.util.List; @@ -30,14 +30,14 @@ import static org.assertj.core.api.Assertions.fail; /** Tests for the {@link ClosableBlockingQueue}. */ -public class ClosableBlockingQueueTest { +class ClosableBlockingQueueTest { // ------------------------------------------------------------------------ // single-threaded unit tests // ------------------------------------------------------------------------ @Test - public void testCreateQueueHashCodeEquals() { + void testCreateQueueHashCodeEquals() { try { ClosableBlockingQueue queue1 = new ClosableBlockingQueue<>(); ClosableBlockingQueue queue2 = new ClosableBlockingQueue<>(22); @@ -91,7 +91,7 @@ public void testCreateQueueHashCodeEquals() { } @Test - public void testCloseEmptyQueue() { + void testCloseEmptyQueue() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); assertThat(queue.isOpen()).isTrue(); @@ -114,7 +114,7 @@ public void testCloseEmptyQueue() { } @Test - public void testCloseNonEmptyQueue() { + void testCloseNonEmptyQueue() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(asList(1, 2, 3)); assertThat(queue.isOpen()).isTrue(); @@ -148,7 +148,7 @@ public void testCloseNonEmptyQueue() { } @Test - public void testPeekAndPoll() { + void testPeekAndPoll() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); @@ -208,7 +208,7 @@ public void testPeekAndPoll() { } @Test - public void testPollBatch() { + void testPollBatch() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); @@ -240,7 +240,7 @@ public void testPollBatch() { } @Test - public void testGetElementBlocking() { + void testGetElementBlocking() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); @@ -297,7 +297,7 @@ public void testGetElementBlocking() { } @Test - public void testGetBatchBlocking() { + void testGetBatchBlocking() { try { ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); @@ -357,7 +357,7 @@ public void testGetBatchBlocking() { // ------------------------------------------------------------------------ @Test - public void notifyOnClose() { + void notifyOnClose() { try { final long oneYear = 365L * 24 * 60 * 60 * 1000; @@ -412,7 +412,7 @@ public void call() throws Exception { @SuppressWarnings("ThrowableResultOfMethodCallIgnored") @Test - public void testMultiThreadedAddGet() { + void testMultiThreadedAddGet() { try { final ClosableBlockingQueue queue = new ClosableBlockingQueue<>(); final AtomicReference pushErrorRef = new AtomicReference<>(); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java index 30e651691..2c0826837 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicPartitionTest.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.lang.reflect.Field; import java.lang.reflect.Modifier; @@ -27,10 +27,10 @@ import static org.assertj.core.api.Assertions.fail; /** Tests for the {@link KafkaTopicPartition}. */ -public class KafkaTopicPartitionTest { +class KafkaTopicPartitionTest { @Test - public void validateUid() { + void validateUid() { Field uidField; try { uidField = KafkaTopicPartition.class.getDeclaredField("serialVersionUID"); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptorTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptorTest.java index bb029d85e..f4bbd8cd8 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptorTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/KafkaTopicsDescriptorTest.java @@ -17,8 +17,10 @@ package org.apache.flink.streaming.connectors.kafka.internals; -import org.junit.Test; -import org.junit.runner.RunWith; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.runners.Parameterized; import java.util.Arrays; @@ -29,10 +31,10 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for the {@link KafkaTopicsDescriptor}. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class KafkaTopicsDescriptorTest { - @Parameterized.Parameters + @Parameters public static Collection data() { return Arrays.asList( new Object[][] { @@ -57,7 +59,7 @@ public KafkaTopicsDescriptorTest( } @Test - public void testIsMatchingTopic() { + void testIsMatchingTopic() { KafkaTopicsDescriptor topicsDescriptor = new KafkaTopicsDescriptor(fixedTopics, topicPattern); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleExactlyOnceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleExactlyOnceITCase.java index 7d37f6c34..4f369d70c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleExactlyOnceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleExactlyOnceITCase.java @@ -27,11 +27,11 @@ import org.apache.flink.streaming.connectors.kafka.testutils.FailingIdentityMapper; import org.apache.flink.streaming.connectors.kafka.testutils.ValidatingExactlyOnceSink; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.UUID; +import java.util.concurrent.TimeUnit; import static org.apache.flink.streaming.api.TimeCharacteristic.EventTime; import static org.apache.flink.streaming.api.TimeCharacteristic.IngestionTime; @@ -39,9 +39,8 @@ import static org.apache.flink.test.util.TestUtils.tryExecute; /** Failure Recovery IT Test for KafkaShuffle. */ -public class KafkaShuffleExactlyOnceITCase extends KafkaShuffleTestBase { - - @Rule public final Timeout timeout = Timeout.millis(600000L); +@Timeout(value = 60000L, unit = TimeUnit.MILLISECONDS) +class KafkaShuffleExactlyOnceITCase extends KafkaShuffleTestBase { /** * Failure Recovery after processing 2/3 data with time characteristic: ProcessingTime. @@ -49,7 +48,7 @@ public class KafkaShuffleExactlyOnceITCase extends KafkaShuffleTestBase { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testFailureRecoveryProcessingTime() throws Exception { + void testFailureRecoveryProcessingTime() throws Exception { testKafkaShuffleFailureRecovery(1000, ProcessingTime); } @@ -59,7 +58,7 @@ public void testFailureRecoveryProcessingTime() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testFailureRecoveryIngestionTime() throws Exception { + void testFailureRecoveryIngestionTime() throws Exception { testKafkaShuffleFailureRecovery(1000, IngestionTime); } @@ -69,7 +68,7 @@ public void testFailureRecoveryIngestionTime() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testFailureRecoveryEventTime() throws Exception { + void testFailureRecoveryEventTime() throws Exception { testKafkaShuffleFailureRecovery(1000, EventTime); } @@ -79,7 +78,7 @@ public void testFailureRecoveryEventTime() throws Exception { *

Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3. */ @Test - public void testAssignedToPartitionFailureRecoveryProcessingTime() throws Exception { + void testAssignedToPartitionFailureRecoveryProcessingTime() throws Exception { testAssignedToPartitionFailureRecovery(500, ProcessingTime); } @@ -89,7 +88,7 @@ public void testAssignedToPartitionFailureRecoveryProcessingTime() throws Except *

Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3. */ @Test - public void testAssignedToPartitionFailureRecoveryIngestionTime() throws Exception { + void testAssignedToPartitionFailureRecoveryIngestionTime() throws Exception { testAssignedToPartitionFailureRecovery(500, IngestionTime); } @@ -99,7 +98,7 @@ public void testAssignedToPartitionFailureRecoveryIngestionTime() throws Excepti *

Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3. */ @Test - public void testAssignedToPartitionFailureRecoveryEventTime() throws Exception { + void testAssignedToPartitionFailureRecoveryEventTime() throws Exception { testAssignedToPartitionFailureRecovery(500, EventTime); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java index 5505bdde3..d274d189e 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleITCase.java @@ -35,9 +35,8 @@ import org.apache.flink.util.PropertiesUtil; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import java.util.ArrayList; import java.util.Collection; @@ -46,6 +45,7 @@ import java.util.Map; import java.util.Properties; import java.util.UUID; +import java.util.concurrent.TimeUnit; import static org.apache.flink.streaming.api.TimeCharacteristic.EventTime; import static org.apache.flink.streaming.api.TimeCharacteristic.IngestionTime; @@ -57,9 +57,8 @@ import static org.assertj.core.api.Assertions.fail; /** Simple End to End Test for Kafka. */ -public class KafkaShuffleITCase extends KafkaShuffleTestBase { - - @Rule public final Timeout timeout = Timeout.millis(600000L); +@Timeout(value = 60000L, unit = TimeUnit.MILLISECONDS) +class KafkaShuffleITCase extends KafkaShuffleTestBase { /** * To test no data is lost or duplicated end-2-end with the default time characteristic: @@ -68,7 +67,7 @@ public class KafkaShuffleITCase extends KafkaShuffleTestBase { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testSimpleProcessingTime() throws Exception { + void testSimpleProcessingTime() throws Exception { testKafkaShuffle(200000, ProcessingTime); } @@ -78,7 +77,7 @@ public void testSimpleProcessingTime() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testSimpleIngestionTime() throws Exception { + void testSimpleIngestionTime() throws Exception { testKafkaShuffle(200000, IngestionTime); } @@ -88,7 +87,7 @@ public void testSimpleIngestionTime() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testSimpleEventTime() throws Exception { + void testSimpleEventTime() throws Exception { testKafkaShuffle(100000, EventTime); } @@ -98,7 +97,7 @@ public void testSimpleEventTime() throws Exception { *

Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3. */ @Test - public void testAssignedToPartitionProcessingTime() throws Exception { + void testAssignedToPartitionProcessingTime() throws Exception { testAssignedToPartition(300000, ProcessingTime); } @@ -108,7 +107,7 @@ public void testAssignedToPartitionProcessingTime() throws Exception { *

Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3. */ @Test - public void testAssignedToPartitionIngestionTime() throws Exception { + void testAssignedToPartitionIngestionTime() throws Exception { testAssignedToPartition(300000, IngestionTime); } @@ -118,7 +117,7 @@ public void testAssignedToPartitionIngestionTime() throws Exception { *

Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3. */ @Test - public void testAssignedToPartitionEventTime() throws Exception { + void testAssignedToPartitionEventTime() throws Exception { testAssignedToPartition(100000, EventTime); } @@ -128,7 +127,7 @@ public void testAssignedToPartitionEventTime() throws Exception { *

Producer Parallelism = 2; Kafka Partition # = 3; Consumer Parallelism = 3. */ @Test - public void testWatermarkIncremental() throws Exception { + void testWatermarkIncremental() throws Exception { testWatermarkIncremental(100000); } @@ -138,7 +137,7 @@ public void testWatermarkIncremental() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testSerDeProcessingTime() throws Exception { + void testSerDeProcessingTime() throws Exception { testRecordSerDe(ProcessingTime); } @@ -149,7 +148,7 @@ public void testSerDeProcessingTime() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testSerDeIngestionTime() throws Exception { + void testSerDeIngestionTime() throws Exception { testRecordSerDe(IngestionTime); } @@ -160,7 +159,7 @@ public void testSerDeIngestionTime() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testSerDeEventTime() throws Exception { + void testSerDeEventTime() throws Exception { testRecordSerDe(EventTime); } @@ -171,7 +170,7 @@ public void testSerDeEventTime() throws Exception { *

Producer Parallelism = 1; Kafka Partition # = 1; Consumer Parallelism = 1. */ @Test - public void testWatermarkBroadcasting() throws Exception { + void testWatermarkBroadcasting() throws Exception { final int numberOfPartitions = 3; final int producerParallelism = 2; final int numElementsPerProducer = 1000; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleTestBase.java index 064aebd7f..9eed87b0b 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/shuffle/KafkaShuffleTestBase.java @@ -40,18 +40,18 @@ import org.apache.flink.test.util.SuccessException; import org.apache.flink.util.Collector; -import org.junit.BeforeClass; +import org.junit.jupiter.api.BeforeAll; import java.util.Random; import static org.apache.flink.streaming.api.TimeCharacteristic.EventTime; /** Base Test Class for KafkaShuffle. */ -public class KafkaShuffleTestBase extends KafkaConsumerTestBase { +class KafkaShuffleTestBase extends KafkaConsumerTestBase { static final long INIT_TIMESTAMP = System.currentTimeMillis(); - @BeforeClass - public static void prepare() throws Exception { + @BeforeAll + protected static void prepare() throws Exception { KafkaProducerTestBase.prepare(); ((KafkaTestEnvironmentImpl) kafkaServer) .setProducerSemantic(FlinkKafkaProducer.Semantic.EXACTLY_ONCE); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java index e8bc9e373..cf54e00ef 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java @@ -32,8 +32,8 @@ import org.apache.flink.table.api.config.OptimizerConfigOptions; import org.apache.kafka.clients.producer.ProducerConfig; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import java.time.Duration; import java.time.ZoneId; @@ -45,17 +45,17 @@ import static org.apache.flink.streaming.connectors.kafka.table.KafkaTableTestUtils.waitingExpectedResults; /** IT cases for Kafka with changelog format for Table API & SQL. */ -public class KafkaChangelogTableITCase extends KafkaTableTestBase { +class KafkaChangelogTableITCase extends KafkaTableTestBase { - @Before - public void before() { + @BeforeEach + void before() { // we have to use single parallelism, // because we will count the messages in sink to terminate the job env.setParallelism(1); } @Test - public void testKafkaDebeziumChangelogSource() throws Exception { + void testKafkaDebeziumChangelogSource() throws Exception { final String topic = "changelog_topic"; createTestTopic(topic, 1, 1); @@ -182,7 +182,7 @@ public void testKafkaDebeziumChangelogSource() throws Exception { } @Test - public void testKafkaCanalChangelogSource() throws Exception { + void testKafkaCanalChangelogSource() throws Exception { final String topic = "changelog_canal"; createTestTopic(topic, 1, 1); @@ -323,7 +323,7 @@ public void testKafkaCanalChangelogSource() throws Exception { } @Test - public void testKafkaMaxwellChangelogSource() throws Exception { + void testKafkaMaxwellChangelogSource() throws Exception { final String topic = "changelog_maxwell"; createTestTopic(topic, 1, 1); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtilTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtilTest.java index e9959087c..182425c65 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtilTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaConnectorOptionsUtilTest.java @@ -23,7 +23,7 @@ import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.types.DataType; -import org.junit.Test; +import org.junit.jupiter.api.Test; import java.util.HashMap; import java.util.Map; @@ -38,10 +38,10 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link KafkaConnectorOptionsUtil}. */ -public class KafkaConnectorOptionsUtilTest { +class KafkaConnectorOptionsUtilTest { @Test - public void testFormatProjection() { + void testFormatProjection() { final DataType dataType = DataTypes.ROW( FIELD("id", INT()), @@ -60,7 +60,7 @@ public void testFormatProjection() { } @Test - public void testMissingKeyFormatProjection() { + void testMissingKeyFormatProjection() { final DataType dataType = ROW(FIELD("id", INT())); final Map options = createTestOptions(); @@ -74,7 +74,7 @@ public void testMissingKeyFormatProjection() { } @Test - public void testInvalidKeyFormatFieldProjection() { + void testInvalidKeyFormatFieldProjection() { final DataType dataType = ROW(FIELD("id", INT()), FIELD("name", STRING())); final Map options = createTestOptions(); options.put("key.fields", "non_existing"); @@ -92,7 +92,7 @@ public void testInvalidKeyFormatFieldProjection() { } @Test - public void testInvalidKeyFormatPrefixProjection() { + void testInvalidKeyFormatPrefixProjection() { final DataType dataType = ROW(FIELD("k_part_1", INT()), FIELD("part_2", STRING()), FIELD("name", STRING())); final Map options = createTestOptions(); @@ -109,7 +109,7 @@ public void testInvalidKeyFormatPrefixProjection() { } @Test - public void testInvalidValueFormatProjection() { + void testInvalidValueFormatProjection() { final DataType dataType = ROW(FIELD("k_id", INT()), FIELD("id", STRING())); final Map options = createTestOptions(); options.put("key.fields", "k_id"); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java index 1246d53a3..23a239766 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaDynamicTableFactoryTest.java @@ -77,7 +77,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.NullSource; @@ -187,7 +187,7 @@ public class KafkaDynamicTableFactoryTest { private static final DataType SCHEMA_DATA_TYPE = SCHEMA.toPhysicalRowDataType(); @Test - public void testTableSource() { + void testTableSource() { final DynamicTableSource actualSource = createTableSource(SCHEMA, getBasicSourceOptions()); final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource; @@ -221,7 +221,7 @@ public void testTableSource() { } @Test - public void testTableSourceWithPattern() { + void testTableSourceWithPattern() { final Map modifiedOptions = getModifiedOptions( getBasicSourceOptions(), @@ -265,7 +265,7 @@ public void testTableSourceWithPattern() { } @Test - public void testTableSourceWithKeyValue() { + void testTableSourceWithKeyValue() { final DynamicTableSource actualSource = createTableSource(SCHEMA, getKeyValueOptions()); final KafkaDynamicSource actualKafkaSource = (KafkaDynamicSource) actualSource; // initialize stateful testing formats @@ -301,7 +301,7 @@ public void testTableSourceWithKeyValue() { } @Test - public void testTableSourceWithKeyValueAndMetadata() { + void testTableSourceWithKeyValueAndMetadata() { final Map options = getKeyValueOptions(); options.put("value.test-format.readable-metadata", "metadata_1:INT, metadata_2:STRING"); @@ -354,7 +354,7 @@ public void testTableSourceWithKeyValueAndMetadata() { } @Test - public void testTableSourceCommitOnCheckpointDisabled() { + void testTableSourceCommitOnCheckpointDisabled() { final Map modifiedOptions = getModifiedOptions( getBasicSourceOptions(), options -> options.remove("properties.group.id")); @@ -387,7 +387,7 @@ public void testTableSourceSetOffsetReset(final String strategyName) { } @Test - public void testTableSourceSetOffsetResetWithException() { + void testTableSourceSetOffsetResetWithException() { String errorStrategy = "errorStrategy"; assertThatThrownBy(() -> testTableSourceSetOffsetReset(errorStrategy)) .isInstanceOf(IllegalArgumentException.class) @@ -430,7 +430,7 @@ private void testSetOffsetResetForStartFromGroupOffsets(String value) { } @Test - public void testBoundedSpecificOffsetsValidate() { + void testBoundedSpecificOffsetsValidate() { final Map modifiedOptions = getModifiedOptions( getBasicSourceOptions(), @@ -446,7 +446,7 @@ public void testBoundedSpecificOffsetsValidate() { } @Test - public void testBoundedSpecificOffsets() { + void testBoundedSpecificOffsets() { testBoundedOffsets( "specific-offsets", options -> { @@ -468,7 +468,7 @@ public void testBoundedSpecificOffsets() { } @Test - public void testBoundedLatestOffset() { + void testBoundedLatestOffset() { testBoundedOffsets( "latest-offset", options -> {}, @@ -492,7 +492,7 @@ public void testBoundedLatestOffset() { } @Test - public void testBoundedGroupOffsets() { + void testBoundedGroupOffsets() { testBoundedOffsets( "group-offsets", options -> {}, @@ -512,7 +512,7 @@ public void testBoundedGroupOffsets() { } @Test - public void testBoundedTimestamp() { + void testBoundedTimestamp() { testBoundedOffsets( "timestamp", options -> { @@ -579,7 +579,7 @@ private void testBoundedOffsets( } @Test - public void testTableSink() { + void testTableSink() { final Map modifiedOptions = getModifiedOptions( getBasicSinkOptions(), @@ -619,7 +619,7 @@ public void testTableSink() { } @Test - public void testTableSinkSemanticTranslation() { + void testTableSinkSemanticTranslation() { final List semantics = Arrays.asList("exactly-once", "at-least-once", "none"); final EncodingFormat> valueEncodingFormat = new EncodingFormatMock(","); @@ -651,7 +651,7 @@ public void testTableSinkSemanticTranslation() { } @Test - public void testTableSinkWithKeyValue() { + void testTableSinkWithKeyValue() { final Map modifiedOptions = getModifiedOptions( getKeyValueOptions(), @@ -694,7 +694,7 @@ public void testTableSinkWithKeyValue() { } @Test - public void testTableSinkWithParallelism() { + void testTableSinkWithParallelism() { final Map modifiedOptions = getModifiedOptions( getBasicSinkOptions(), options -> options.put("sink.parallelism", "100")); @@ -728,7 +728,7 @@ public void testTableSinkWithParallelism() { } @Test - public void testTableSinkAutoCompleteSchemaRegistrySubject() { + void testTableSinkAutoCompleteSchemaRegistrySubject() { // only format verifyEncoderSubject( options -> { @@ -876,7 +876,7 @@ private SerializationSchema createDebeziumAvroSerSchema( // -------------------------------------------------------------------------------------------- @Test - public void testSourceTableWithTopicAndTopicPattern() { + void testSourceTableWithTopicAndTopicPattern() { assertThatThrownBy( () -> { final Map modifiedOptions = @@ -897,7 +897,7 @@ public void testSourceTableWithTopicAndTopicPattern() { } @Test - public void testMissingStartupTimestamp() { + void testMissingStartupTimestamp() { assertThatThrownBy( () -> { final Map modifiedOptions = @@ -917,7 +917,7 @@ public void testMissingStartupTimestamp() { } @Test - public void testMissingSpecificOffsets() { + void testMissingSpecificOffsets() { assertThatThrownBy( () -> { final Map modifiedOptions = @@ -938,7 +938,7 @@ public void testMissingSpecificOffsets() { } @Test - public void testInvalidSinkPartitioner() { + void testInvalidSinkPartitioner() { assertThatThrownBy( () -> { final Map modifiedOptions = @@ -956,7 +956,7 @@ public void testInvalidSinkPartitioner() { } @Test - public void testInvalidRoundRobinPartitionerWithKeyFields() { + void testInvalidRoundRobinPartitionerWithKeyFields() { assertThatThrownBy( () -> { final Map modifiedOptions = @@ -976,7 +976,7 @@ public void testInvalidRoundRobinPartitionerWithKeyFields() { } @Test - public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { + void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { assertThatThrownBy( () -> { final Map modifiedOptions = @@ -1002,7 +1002,7 @@ public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { } @Test - public void testSinkWithTopicListOrTopicPattern() { + void testSinkWithTopicListOrTopicPattern() { Map modifiedOptions = getModifiedOptions( getBasicSinkOptions(), @@ -1039,7 +1039,7 @@ public void testSinkWithTopicListOrTopicPattern() { } @Test - public void testPrimaryKeyValidation() { + void testPrimaryKeyValidation() { final ResolvedSchema pkSchema = new ResolvedSchema( SCHEMA.getColumns(), @@ -1098,7 +1098,7 @@ public void testPrimaryKeyValidation() { } @Test - public void testDiscoverPartitionByDefault() { + void testDiscoverPartitionByDefault() { Map tableSourceOptions = getModifiedOptions( getBasicSourceOptions(), @@ -1136,7 +1136,7 @@ public void testDiscoverPartitionByDefault() { } @Test - public void testDisableDiscoverPartition() { + void testDisableDiscoverPartition() { Map tableSourceOptions = getModifiedOptions( getBasicSourceOptions(), diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java index 409acd977..a28d44af6 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java @@ -35,16 +35,18 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.table.utils.EncodingUtils; import org.apache.flink.test.util.SuccessException; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.types.Row; import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.time.Instant; @@ -76,21 +78,21 @@ import static org.assertj.core.api.HamcrestCondition.matching; /** Basic IT cases for the Kafka table source and sink. */ -@RunWith(Parameterized.class) -public class KafkaTableITCase extends KafkaTableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +class KafkaTableITCase extends KafkaTableTestBase { private static final String JSON_FORMAT = "json"; private static final String AVRO_FORMAT = "avro"; private static final String CSV_FORMAT = "csv"; - @Parameterized.Parameter public String format; + @Parameter public String format; - @Parameterized.Parameters(name = "format = {0}") + @Parameters(name = "format = {0}") public static Collection parameters() { return Arrays.asList(JSON_FORMAT, AVRO_FORMAT, CSV_FORMAT); } - @Before + @BeforeEach public void before() { // we have to use single parallelism, // because we will count the messages in sink to terminate the job @@ -98,7 +100,7 @@ public void before() { } @Test - public void testKafkaSourceSink() throws Exception { + void testKafkaSourceSink() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "tstopic_" + format + "_" + UUID.randomUUID(); @@ -189,7 +191,7 @@ public void testKafkaSourceSink() throws Exception { } @Test - public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { + void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "bounded_" + format + "_" + UUID.randomUUID(); @@ -243,7 +245,7 @@ public void testKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { } @Test - public void testKafkaSourceSinkWithBoundedTimestamp() throws Exception { + void testKafkaSourceSinkWithBoundedTimestamp() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "bounded_" + format + "_" + UUID.randomUUID(); @@ -300,7 +302,7 @@ public void testKafkaSourceSinkWithBoundedTimestamp() throws Exception { } @Test - public void testKafkaTableWithMultipleTopics() throws Exception { + void testKafkaTableWithMultipleTopics() throws Exception { // ---------- create source and sink tables ------------------- String tableTemp = "create table %s (\n" @@ -393,7 +395,7 @@ public void testKafkaTableWithMultipleTopics() throws Exception { } @Test - public void testKafkaSourceSinkWithMetadata() throws Exception { + void testKafkaSourceSinkWithMetadata() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "metadata_topic_" + format + "_" + UUID.randomUUID(); @@ -485,7 +487,7 @@ public void testKafkaSourceSinkWithMetadata() throws Exception { } @Test - public void testKafkaSourceSinkWithKeyAndPartialValue() throws Exception { + void testKafkaSourceSinkWithKeyAndPartialValue() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "key_partial_value_topic_" + format + "_" + UUID.randomUUID(); @@ -566,7 +568,7 @@ public void testKafkaSourceSinkWithKeyAndPartialValue() throws Exception { } @Test - public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { + void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "key_full_value_topic_" + format + "_" + UUID.randomUUID(); @@ -644,7 +646,7 @@ public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { } @Test - public void testKafkaTemporalJoinChangelog() throws Exception { + void testKafkaTemporalJoinChangelog() throws Exception { // Set the session time zone to UTC, because the next `METADATA FROM // 'value.source.timestamp'` DDL // will use the session time zone when convert the changelog time from milliseconds to @@ -787,7 +789,7 @@ private void initialProductChangelog(String topic, String bootstraps) throws Exc } @Test - public void testPerPartitionWatermarkKafka() throws Exception { + void testPerPartitionWatermarkKafka() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "per_partition_watermark_topic_" + format + "_" + UUID.randomUUID(); @@ -877,7 +879,7 @@ public void testPerPartitionWatermarkKafka() throws Exception { } @Test - public void testPerPartitionWatermarkWithIdleSource() throws Exception { + void testPerPartitionWatermarkWithIdleSource() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "idle_partition_watermark_topic_" + format + "_" + UUID.randomUUID(); @@ -952,7 +954,7 @@ public void testPerPartitionWatermarkWithIdleSource() throws Exception { } @Test - public void testLatestOffsetStrategyResume() throws Exception { + void testLatestOffsetStrategyResume() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "latest_offset_resume_topic_" + format + "_" + UUID.randomUUID(); @@ -1084,17 +1086,17 @@ public void testLatestOffsetStrategyResume() throws Exception { } @Test - public void testStartFromGroupOffsetsLatest() throws Exception { + void testStartFromGroupOffsetsLatest() throws Exception { testStartFromGroupOffsets("latest"); } @Test - public void testStartFromGroupOffsetsEarliest() throws Exception { + void testStartFromGroupOffsetsEarliest() throws Exception { testStartFromGroupOffsets("earliest"); } @Test - public void testStartFromGroupOffsetsNone() { + void testStartFromGroupOffsetsNone() { Assertions.assertThatThrownBy(() -> testStartFromGroupOffsetsWithNoneResetStrategy()) .satisfies(FlinkAssertions.anyCauseMatches(NoOffsetForPartitionException.class)); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java index f80a54fc9..0ca70cd49 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableTestBase.java @@ -35,9 +35,9 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.StringDeserializer; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.testcontainers.junit.jupiter.Container; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.testcontainers.containers.KafkaContainer; @@ -63,7 +63,7 @@ public abstract class KafkaTableTestBase extends AbstractTestBase { private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; private static final int zkTimeoutMills = 30000; - @ClassRule + @Container public static final KafkaContainer KAFKA_CONTAINER = new KafkaContainer(DockerImageName.parse(DockerImageVersions.KAFKA)) { @Override @@ -87,8 +87,8 @@ protected void doStart() { // Timer for scheduling logging task if the test hangs private final Timer loggingTimer = new Timer("Debug Logging Timer"); - @Before - public void setup() { + @BeforeEach + void setup() { env = StreamExecutionEnvironment.getExecutionEnvironment(); tEnv = StreamTableEnvironment.create(env); env.getConfig().setRestartStrategy(RestartStrategies.noRestart()); @@ -107,8 +107,8 @@ public void setup() { }); } - @After - public void after() { + @AfterEach + void after() { // Cancel timer for debug logging cancelTimeoutLogger(); } diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java index 0640b9ad7..a44ad1ef9 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java @@ -33,7 +33,10 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.junit.Test; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -55,9 +58,9 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link ReducingUpsertWriter}. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class ReducingUpsertWriterTest { - @Parameterized.Parameters(name = "object reuse = {0}") + @Parameters(name = "object reuse = {0}") public static Object[] enableObjectReuse() { return new Boolean[] {true, false}; } @@ -150,7 +153,7 @@ public ReducingUpsertWriterTest(boolean enableObjectReuse) { } @Test - public void testWriteData() throws Exception { + void testWriteData() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); @@ -217,7 +220,7 @@ public void testWriteData() throws Exception { } @Test - public void testFlushDataWhenCheckpointing() throws Exception { + void testFlushDataWhenCheckpointing() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); // write all data, there should be 3 records are still buffered @@ -265,7 +268,7 @@ public void testFlushDataWhenCheckpointing() throws Exception { } @Test - public void testWriteDataWithNullTimestamp() throws Exception { + void testWriteDataWithNullTimestamp() throws Exception { final MockedSinkWriter writer = new MockedSinkWriter(); final ReducingUpsertWriter bufferedWriter = createBufferedWriter(writer); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java index 15c740d21..41230bf77 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaDynamicTableFactoryTest.java @@ -66,13 +66,11 @@ import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.util.TestLogger; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; import java.util.Arrays; import java.util.Collections; @@ -91,7 +89,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link UpsertKafkaDynamicTableFactory}. */ -public class UpsertKafkaDynamicTableFactoryTest extends TestLogger { +class UpsertKafkaDynamicTableFactoryTest { private static final String SOURCE_TOPIC = "sourceTopic_1"; @@ -148,10 +146,8 @@ public class UpsertKafkaDynamicTableFactoryTest extends TestLogger { new TestFormatFactory.DecodingFormatMock( ",", true, ChangelogMode.insertOnly(), Collections.emptyMap()); - @Rule public ExpectedException thrown = ExpectedException.none(); - @Test - public void testTableSource() { + void testTableSource() { final DataType producedDataType = SOURCE_SCHEMA.toPhysicalRowDataType(); // Construct table source using options and table source factory final DynamicTableSource actualSource = @@ -176,7 +172,7 @@ public void testTableSource() { } @Test - public void testTableSink() { + void testTableSink() { // Construct table sink using options and table sink factory. final Map modifiedOptions = getModifiedOptions( @@ -217,7 +213,7 @@ public void testTableSink() { @SuppressWarnings("rawtypes") @Test - public void testBufferedTableSink() { + void testBufferedTableSink() { // Construct table sink using options and table sink factory. final DynamicTableSink actualSink = createTableSink( @@ -274,7 +270,7 @@ public void testBufferedTableSink() { } @Test - public void testTableSinkWithParallelism() { + void testTableSinkWithParallelism() { final Map modifiedOptions = getModifiedOptions( getFullSinkOptions(), @@ -310,7 +306,7 @@ public void testTableSinkWithParallelism() { } @Test - public void testTableSinkAutoCompleteSchemaRegistrySubject() { + void testTableSinkAutoCompleteSchemaRegistrySubject() { // value.format + key.format verifyEncoderSubject( options -> { @@ -420,7 +416,7 @@ private SerializationSchema createConfluentAvroSerSchema( // -------------------------------------------------------------------------------------------- @Test - public void testBoundedSpecificOffsetsValidate() { + void testBoundedSpecificOffsetsValidate() { final Map options = getFullSourceOptions(); options.put( KafkaConnectorOptions.SCAN_BOUNDED_MODE.key(), @@ -434,7 +430,7 @@ public void testBoundedSpecificOffsetsValidate() { } @Test - public void testBoundedSpecificOffsets() { + void testBoundedSpecificOffsets() { testBoundedOffsets( ScanBoundedMode.SPECIFIC_OFFSETS, options -> { @@ -456,7 +452,7 @@ public void testBoundedSpecificOffsets() { } @Test - public void testBoundedLatestOffset() { + void testBoundedLatestOffset() { testBoundedOffsets( ScanBoundedMode.LATEST_OFFSET, options -> {}, @@ -480,7 +476,7 @@ public void testBoundedLatestOffset() { } @Test - public void testBoundedGroupOffsets() { + void testBoundedGroupOffsets() { testBoundedOffsets( ScanBoundedMode.GROUP_OFFSETS, options -> { @@ -502,7 +498,7 @@ public void testBoundedGroupOffsets() { } @Test - public void testBoundedTimestamp() { + void testBoundedTimestamp() { testBoundedOffsets( ScanBoundedMode.TIMESTAMP, options -> { @@ -552,10 +548,9 @@ public void testBoundedTimestamp() { // -------------------------------------------------------------------------------------------- @Test - public void testCreateSourceTableWithoutPK() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( + void testCreateSourceTableWithoutPK() { + Assertions.assertThrows(ValidationException.class, + () -> containsCause( new ValidationException( "'upsert-kafka' tables require to define a PRIMARY KEY constraint. " + "The PRIMARY KEY specifies which columns should be read from or write to the Kafka message key. " @@ -570,10 +565,9 @@ public void testCreateSourceTableWithoutPK() { } @Test - public void testCreateSinkTableWithoutPK() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( + void testCreateSinkTableWithoutPK() { + Assertions.assertThrows(ValidationException.class, + () -> containsCause( new ValidationException( "'upsert-kafka' tables require to define a PRIMARY KEY constraint. " + "The PRIMARY KEY specifies which columns should be read from or write to the Kafka message key. " @@ -587,10 +581,9 @@ public void testCreateSinkTableWithoutPK() { } @Test - public void testSerWithCDCFormatAsValue() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( + void testSerWithCDCFormatAsValue() { + Assertions.assertThrows(ValidationException.class, + () -> containsCause( new ValidationException( String.format( "'upsert-kafka' connector doesn't support '%s' as value format, " @@ -612,10 +605,9 @@ public void testSerWithCDCFormatAsValue() { } @Test - public void testDeserWithCDCFormatAsValue() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( + void testDeserWithCDCFormatAsValue() { + Assertions.assertThrows(ValidationException.class, + () -> containsCause( new ValidationException( String.format( "'upsert-kafka' connector doesn't support '%s' as value format, " @@ -637,10 +629,9 @@ public void testDeserWithCDCFormatAsValue() { } @Test - public void testInvalidSinkBufferFlush() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( + void testInvalidSinkBufferFlush() { + Assertions.assertThrows(ValidationException.class, + () -> containsCause( new ValidationException( "'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' " + "must be set to be greater than zero together to enable" @@ -656,10 +647,9 @@ public void testInvalidSinkBufferFlush() { } @Test - public void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { - thrown.expect(ValidationException.class); - thrown.expect( - containsCause( + void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { + Assertions.assertThrows(ValidationException.class, + () -> containsCause( new ValidationException( "sink.transactional-id-prefix must be specified when using DeliveryGuarantee.EXACTLY_ONCE."))); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java index 1a6bf7e13..96e615623 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/UpsertKafkaTableITCase.java @@ -24,12 +24,13 @@ import org.apache.flink.table.api.TableResult; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.table.utils.LegacyRowResource; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.apache.flink.types.Row; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.time.Duration; import java.time.LocalDateTime; @@ -56,27 +57,25 @@ import static org.assertj.core.api.HamcrestCondition.matching; /** Upsert-kafka IT cases. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class UpsertKafkaTableITCase extends KafkaTableTestBase { private static final String JSON_FORMAT = "json"; private static final String CSV_FORMAT = "csv"; private static final String AVRO_FORMAT = "avro"; - @Parameterized.Parameter public String format; + @Parameter public String format; - @Parameterized.Parameters(name = "format = {0}") + @Parameters(name = "format = {0}") public static Object[] parameters() { return new Object[] {JSON_FORMAT, CSV_FORMAT, AVRO_FORMAT}; } - @Rule public final LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE; - private static final String USERS_TOPIC = "users"; private static final String WORD_COUNT_TOPIC = "word_count"; @Test - public void testAggregate() throws Exception { + void testAggregate() throws Exception { String topic = WORD_COUNT_TOPIC + "_" + format; createTestTopic(topic, 4, 1); // ------------- test --------------- @@ -87,7 +86,7 @@ public void testAggregate() throws Exception { } @Test - public void testTemporalJoin() throws Exception { + void testTemporalJoin() throws Exception { String topic = USERS_TOPIC + "_" + format; createTestTopic(topic, 2, 1); // ------------- test --------------- @@ -110,7 +109,7 @@ public void testTemporalJoin() throws Exception { } @Test - public void testBufferedUpsertSink() throws Exception { + void testBufferedUpsertSink() throws Exception { final String topic = "buffered_upsert_topic_" + format; createTestTopic(topic, 1, 1); String bootstraps = getBootstrapServers(); @@ -199,7 +198,7 @@ public void testBufferedUpsertSink() throws Exception { } @Test - public void testBufferedUpsertSinkWithoutAssigningWatermark() throws Exception { + void testBufferedUpsertSinkWithoutAssigningWatermark() throws Exception { final String topic = "buffered_upsert_topic_without_assigning_watermark_" + format; createTestTopic(topic, 1, 1); String bootstraps = getBootstrapServers(); @@ -264,7 +263,7 @@ public void testBufferedUpsertSinkWithoutAssigningWatermark() throws Exception { } @Test - public void testSourceSinkWithKeyAndPartialValue() throws Exception { + void testSourceSinkWithKeyAndPartialValue() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "key_partial_value_topic_" + format; @@ -362,7 +361,7 @@ public void testSourceSinkWithKeyAndPartialValue() throws Exception { } @Test - public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { + void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { // we always use a different topic name for each parameterized topic, // in order to make sure the topic can be created. final String topic = "key_full_value_topic_" + format; @@ -457,7 +456,7 @@ public void testKafkaSourceSinkWithKeyAndFullValue() throws Exception { } @Test - public void testUpsertKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { + void testUpsertKafkaSourceSinkWithBoundedSpecificOffsets() throws Exception { final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); createTestTopic(topic, 1, 1); @@ -510,7 +509,7 @@ public void testUpsertKafkaSourceSinkWithBoundedSpecificOffsets() throws Excepti } @Test - public void testUpsertKafkaSourceSinkWithBoundedTimestamp() throws Exception { + void testUpsertKafkaSourceSinkWithBoundedTimestamp() throws Exception { final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); createTestTopic(topic, 1, 1); @@ -596,7 +595,7 @@ public void testUpsertKafkaSourceSinkWithBoundedTimestamp() throws Exception { * results. */ @Test - public void testUpsertKafkaSourceSinkWithZeroLengthBoundedness() throws Exception { + void testUpsertKafkaSourceSinkWithZeroLengthBoundedness() throws Exception { final String topic = "bounded_upsert_" + format + "_" + UUID.randomUUID(); createTestTopic(topic, 1, 1); From 18b67db4b0d3de500103a114aac0a816a9067652 Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 09:41:56 +0800 Subject: [PATCH 2/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../flink/connector/kafka/source/KafkaSourceITCase.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java index dac1893a2..101c0eebe 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceITCase.java @@ -167,7 +167,7 @@ public void testBasicRead(boolean enableObjectReuse) throws Exception { executeAndVerify(env, stream); } - @Test + @Test void testValueOnlyDeserializer() throws Exception { KafkaSource source = KafkaSource.builder() @@ -264,7 +264,7 @@ public void testBasicReadWithoutGroupId(boolean enableObjectReuse) throws Except executeAndVerify(env, stream); } - @Test + @Test void testPerPartitionWatermark() throws Throwable { String watermarkTopic = "watermarkTestTopic-" + UUID.randomUUID(); KafkaSourceTestEnv.createTestTopic(watermarkTopic, 2, 1); @@ -311,7 +311,7 @@ public void processElement( env.execute(); } - @Test + @Test void testConsumingEmptyTopic() throws Throwable { String emptyTopic = "emptyTopic-" + UUID.randomUUID(); KafkaSourceTestEnv.createTestTopic(emptyTopic, 3, 1); @@ -336,7 +336,7 @@ void testConsumingEmptyTopic() throws Throwable { } } - @Test + @Test void testConsumingTopicWithEmptyPartitions() throws Throwable { String topicWithEmptyPartitions = "topicWithEmptyPartitions-" + UUID.randomUUID(); KafkaSourceTestEnv.createTestTopic( From a988726a1518a47c3986ad7b2da91cbfebd9fc6f Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 09:51:19 +0800 Subject: [PATCH 3/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../deserializer/KafkaRecordDeserializationSchemaTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java index 31ea90d7a..af53b0c3c 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/reader/deserializer/KafkaRecordDeserializationSchemaTest.java @@ -44,7 +44,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Unit tests for KafkaRecordDeserializationSchema. */ -public class KafkaRecordDeserializationSchemaTest { +class KafkaRecordDeserializationSchemaTest { private static final ObjectMapper OBJECT_MAPPER = JacksonMapperFactory.createObjectMapper(); @@ -52,8 +52,8 @@ public class KafkaRecordDeserializationSchemaTest { private static Map configuration; private static boolean isKeyDeserializer; - @Before - public void setUp() { + @BeforeEach + void setUp() { configurableConfiguration = new HashMap<>(1); configuration = new HashMap<>(1); isKeyDeserializer = false; From d2bfcc9df83d9d07a7b3be111e475ddb3a189b71 Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 09:56:31 +0800 Subject: [PATCH 4/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../connectors/kafka/KafkaShortRetentionTestBase.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index 17ac62602..22d3877b6 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -36,13 +36,11 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.io.TempDir; import org.testcontainers.junit.jupiter.Container; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Serializable; -import java.nio.file.Path; import java.util.Properties; import java.util.UUID; @@ -53,7 +51,7 @@ * can make sure our consumer is properly handling cases where we run into out of offset errors */ @SuppressWarnings("serial") -public class KafkaShortRetentionTestBase implements Serializable { +class KafkaShortRetentionTestBase implements Serializable { protected static final Logger LOG = LoggerFactory.getLogger(KafkaShortRetentionTestBase.class); @@ -75,8 +73,6 @@ public class KafkaShortRetentionTestBase implements Serializable { .setNumberSlotsPerTaskManager(TM_SLOTS) .build()); - @TempDir public Path tempFolder; - protected static Properties secureProps = new Properties(); private static Configuration getConfiguration() { From da0001aa37f33e7e66a0a645c854a2a0cc24223d Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 10:00:51 +0800 Subject: [PATCH 5/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../apache/flink/streaming/connectors/kafka/KafkaTestBase.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java index 3b0e0ae79..3d4ade2b4 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBase.java @@ -91,8 +91,6 @@ public abstract class KafkaTestBase extends TestLogger { public static KafkaTestEnvironment kafkaServer; - @TempDir public static File temporaryFolder; - public static Properties secureProps = new Properties(); // ------------------------------------------------------------------------ From cde9555204f8d860f6fd137f1a99d2eb14a8ab13 Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 10:27:17 +0800 Subject: [PATCH 6/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../connectors/kafka/FlinkKafkaConsumerITCase.java | 6 +++--- .../connectors/kafka/KafkaShortRetentionTestBase.java | 6 +++--- .../streaming/connectors/kafka/KafkaTestBaseWithFlink.java | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java index 74069be27..c66e11e19 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java @@ -31,7 +31,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.junit.jupiter.api.AfterAll; @@ -52,8 +52,8 @@ class FlinkKafkaConsumerITCase { private static final String TOPIC1 = "FlinkKafkaConsumerITCase_topic1"; @Container - public static final MiniClusterWithClientResource MINI_CLUSTER = - new MiniClusterWithClientResource( + public static final MiniClusterExtension MINI_CLUSTER = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setConfiguration(new Configuration()) .build()); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index 22d3877b6..85d4627f0 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -30,7 +30,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.util.InstantiationUtil; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -65,8 +65,8 @@ class KafkaShortRetentionTestBase implements Serializable { private static Properties standardProps; @Container - public static MiniClusterWithClientResource flink = - new MiniClusterWithClientResource( + public static MiniClusterExtension flink = + new MiniClusterExtension ( new MiniClusterResourceConfiguration.Builder() .setConfiguration(getConfiguration()) .setNumberTaskManagers(NUM_TMS) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java index 163b25cb2..ba00ddf51 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.testcontainers.junit.jupiter.Container; @@ -31,8 +31,8 @@ public abstract class KafkaTestBaseWithFlink extends KafkaTestBase { protected static final int TM_SLOTS = 8; @Container - public static MiniClusterWithClientResource flink = - new MiniClusterWithClientResource( + public static MiniClusterExtension flink = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setConfiguration(getFlinkConfiguration()) .setNumberTaskManagers(NUM_TMS) From 8c79057c514a7fe1447387da983772d415262a6b Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 10:34:37 +0800 Subject: [PATCH 7/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../connectors/kafka/FlinkKafkaConsumerITCase.java | 6 +++--- .../connectors/kafka/KafkaShortRetentionTestBase.java | 6 +++--- .../streaming/connectors/kafka/KafkaTestBaseWithFlink.java | 6 +++--- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java index c66e11e19..74069be27 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaConsumerITCase.java @@ -31,7 +31,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.junit.jupiter.api.AfterAll; @@ -52,8 +52,8 @@ class FlinkKafkaConsumerITCase { private static final String TOPIC1 = "FlinkKafkaConsumerITCase_topic1"; @Container - public static final MiniClusterExtension MINI_CLUSTER = - new MiniClusterExtension( + public static final MiniClusterWithClientResource MINI_CLUSTER = + new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() .setConfiguration(new Configuration()) .build()); diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java index 85d4627f0..22d3877b6 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaShortRetentionTestBase.java @@ -30,7 +30,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.DiscardingSink; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; -import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.util.InstantiationUtil; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -65,8 +65,8 @@ class KafkaShortRetentionTestBase implements Serializable { private static Properties standardProps; @Container - public static MiniClusterExtension flink = - new MiniClusterExtension ( + public static MiniClusterWithClientResource flink = + new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() .setConfiguration(getConfiguration()) .setNumberTaskManagers(NUM_TMS) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java index ba00ddf51..163b25cb2 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTestBaseWithFlink.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.connectors.kafka; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.MiniClusterWithClientResource; import org.testcontainers.junit.jupiter.Container; @@ -31,8 +31,8 @@ public abstract class KafkaTestBaseWithFlink extends KafkaTestBase { protected static final int TM_SLOTS = 8; @Container - public static MiniClusterExtension flink = - new MiniClusterExtension( + public static MiniClusterWithClientResource flink = + new MiniClusterWithClientResource( new MiniClusterResourceConfiguration.Builder() .setConfiguration(getFlinkConfiguration()) .setNumberTaskManagers(NUM_TMS) From 49b44bc430b54e75a52f65ad726beb2052f49ab4 Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Fri, 10 Nov 2023 09:52:50 +0800 Subject: [PATCH 8/8] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../kafka/internals/AbstractFetcherWatermarksTest.java | 2 -- .../kafka/internals/AbstractPartitionDiscovererTest.java | 2 -- .../connectors/kafka/table/ReducingUpsertWriterTest.java | 2 -- 3 files changed, 6 deletions(-) diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java index c393bd306..f07fa2349 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractFetcherWatermarksTest.java @@ -37,9 +37,7 @@ import org.apache.flink.util.SerializedValue; import org.junit.jupiter.api.Test; -import org.junit.experimental.runners.Enclosed; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.runner.RunWith; import javax.annotation.Nonnull; import javax.annotation.Nullable; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java index 349941499..4b72e7034 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/internals/AbstractPartitionDiscovererTest.java @@ -24,8 +24,6 @@ import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.util.ArrayList; import java.util.Arrays; diff --git a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java index a44ad1ef9..1186c7202 100644 --- a/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java +++ b/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/ReducingUpsertWriterTest.java @@ -37,8 +37,6 @@ import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; import java.io.IOException; import java.time.Instant;