From f345a09443502bd77d21a855926e6f8133dde724 Mon Sep 17 00:00:00 2001 From: morazow Date: Wed, 19 Jun 2024 16:21:47 +0200 Subject: [PATCH 01/15] [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 | 193 +++++++++--------- .../kafka/sink/KafkaTransactionLogITCase.java | 19 +- .../kafka/sink/KafkaWriterITCase.java | 187 ++++++++--------- .../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 | 62 +++--- .../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, 861 insertions(+), 824 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 9cc80518a..d4d2ac444 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 @@ -58,23 +58,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 +90,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 +112,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 +128,20 @@ public class KafkaSinkITCase extends TestLogger { private SharedReference failed; private SharedReference lastCheckpointedRecord; - @ClassRule - public static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) - .withEmbeddedZookeeper() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); + @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 +149,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 +164,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); } @@ -172,43 +179,41 @@ class IntegrationTests extends SinkTestSuiteBase { // Defines external system @TestExternalSystem - DefaultContainerizedExternalSystem kafka = - DefaultContainerizedExternalSystem.builder() - .fromContainer( - new KafkaContainer( - DockerImageName.parse(DockerImageVersions.KAFKA))) - .build(); + DefaultContainerizedExternalSystem kafka = DefaultContainerizedExternalSystem.builder() + .fromContainer( + new KafkaContainer( + DockerImageName.parse(DockerImageVersions.KAFKA))) + .build(); @SuppressWarnings("unused") @TestSemantics - CheckpointingMode[] semantics = - new CheckpointingMode[] { - CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE - }; + CheckpointingMode[] semantics = new CheckpointingMode[] { + CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE + }; @SuppressWarnings("unused") @TestContext - KafkaSinkExternalContextFactory sinkContext = - new KafkaSinkExternalContextFactory(kafka.getContainer(), Collections.emptyList()); + KafkaSinkExternalContextFactory sinkContext = new KafkaSinkExternalContextFactory(kafka.getContainer(), + Collections.emptyList()); } @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,38 +221,36 @@ public void testRecoveryWithAtLeastOnceGuarantee() throws Exception { } @Test - public void testRecoveryWithExactlyOnceGuarantee() throws Exception { + void testRecoveryWithExactlyOnceGuarantee() throws Exception { testRecoveryWithAssertion( DeliveryGuarantee.EXACTLY_ONCE, 1, - (records) -> - assertThat(records) - .contains( - (LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)))); + (records) -> assertThat(records) + .contains( + (LongStream.range(1, lastCheckpointedRecord.get().get() + 1) + .boxed() + .toArray(Long[]::new)))); } @Test - public void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Exception { + void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Exception { testRecoveryWithAssertion( DeliveryGuarantee.EXACTLY_ONCE, 2, - (records) -> - assertThat(records) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new))); + (records) -> assertThat(records) + .contains( + LongStream.range(1, lastCheckpointedRecord.get().get() + 1) + .boxed() + .toArray(Long[]::new))); } @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( @@ -269,8 +272,7 @@ public void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Excep failed.get().set(true); executeWithMapper( new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, "newPrefix"); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, true); + final List> collectedRecords = drainAllRecordsFromTopic(topic, true); assertThat(deserializeValues(collectedRecords)) .contains( LongStream.range(1, lastCheckpointedRecord.get().get() + 1) @@ -279,7 +281,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); @@ -296,8 +298,7 @@ public void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exce failed.get().set(true); executeWithMapper( new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, null); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, true); + final List> collectedRecords = drainAllRecordsFromTopic(topic, true); assertThat(deserializeValues(collectedRecords)) .contains( LongStream.range(1, lastCheckpointedRecord.get().get() + 1) @@ -315,15 +316,14 @@ private void executeWithMapper( env.setRestartStrategy(RestartStrategies.noRestart()); final DataStreamSource source = env.fromSequence(1, 10); final DataStream stream = source.map(mapper); - final KafkaSinkBuilder builder = - new KafkaSinkBuilder() - .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) - .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) - .setRecordSerializer( - KafkaRecordSerializationSchema.builder() - .setTopic(topic) - .setValueSerializationSchema(new RecordSerializer()) - .build()); + final KafkaSinkBuilder builder = new KafkaSinkBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) + .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) + .setRecordSerializer( + KafkaRecordSerializationSchema.builder() + .setTopic(topic) + .setValueSerializationSchema(new RecordSerializer()) + .build()); if (transactionalIdPrefix == null) { transactionalIdPrefix = "kafka-sink"; } @@ -342,8 +342,7 @@ private void testRecoveryWithAssertion( env.enableCheckpointing(300L); env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrentCheckpoints); DataStreamSource source = env.fromSequence(1, 10); - DataStream stream = - source.map(new FailingCheckpointMapper(failed, lastCheckpointedRecord)); + DataStream stream = source.map(new FailingCheckpointMapper(failed, lastCheckpointedRecord)); stream.sinkTo( new KafkaSinkBuilder() @@ -358,8 +357,8 @@ private void testRecoveryWithAssertion( .build()); env.execute(); - final List> collectedRecords = - drainAllRecordsFromTopic(topic, guarantee == DeliveryGuarantee.EXACTLY_ONCE); + final List> collectedRecords = drainAllRecordsFromTopic(topic, + guarantee == DeliveryGuarantee.EXACTLY_ONCE); recordsAssertion.accept(deserializeValues(collectedRecords)); checkProducerLeak(); } @@ -369,10 +368,9 @@ private void writeRecordsToKafka( throws Exception { final StreamExecutionEnvironment env = new LocalStreamEnvironment(); env.enableCheckpointing(100L); - final DataStream source = - env.addSource( - new InfiniteIntegerSource( - emittedRecordsCount, emittedRecordsWithCheckpoint)); + final DataStream source = env.addSource( + new InfiniteIntegerSource( + emittedRecordsCount, emittedRecordsWithCheckpoint)); source.sinkTo( new KafkaSinkBuilder() .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) @@ -386,9 +384,8 @@ private void writeRecordsToKafka( .build()); env.execute(); - final List> collectedRecords = - drainAllRecordsFromTopic( - topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); + final List> collectedRecords = drainAllRecordsFromTopic( + topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); final long recordsCount = expectedRecords.get().get(); assertThat(recordsCount).isEqualTo(collectedRecords.size()); assertThat(deserializeValues(collectedRecords)) @@ -423,10 +420,9 @@ private static Properties getKafkaClientConfiguration() { private void createTestTopic(String topic, int numPartitions, short replicationFactor) throws ExecutionException, InterruptedException, TimeoutException { - final CreateTopicsResult result = - admin.createTopics( - Collections.singletonList( - new NewTopic(topic, numPartitions, replicationFactor))); + final CreateTopicsResult result = admin.createTopics( + Collections.singletonList( + new NewTopic(topic, numPartitions, replicationFactor))); result.all().get(); } @@ -454,8 +450,8 @@ public byte[] serialize(Long element) { private static class FailAsyncCheckpointMapper implements MapFunction, CheckpointedFunction { - private static final ListStateDescriptor stateDescriptor = - new ListStateDescriptor<>("test-state", new SlowSerializer()); + private static final ListStateDescriptor stateDescriptor = new ListStateDescriptor<>("test-state", + new SlowSerializer()); private int failAfterCheckpoint; private ListState state; @@ -534,7 +530,8 @@ public Integer deserialize(Integer reuse, DataInputView source) throws IOExcepti } @Override - public void copy(DataInputView source, DataOutputView target) throws IOException {} + public void copy(DataInputView source, DataOutputView target) throws IOException { + } @Override public TypeSerializerSnapshot snapshotConfiguration() { @@ -598,16 +595,16 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } @Override - public void initializeState(FunctionInitializationContext context) throws Exception {} + public void initializeState(FunctionInitializationContext context) throws Exception { + } } private void checkProducerLeak() throws InterruptedException { List> leaks = null; for (int tries = 0; tries < 10; tries++) { - leaks = - Thread.getAllStackTraces().entrySet().stream() - .filter(this::findAliveKafkaThread) - .collect(Collectors.toList()); + leaks = Thread.getAllStackTraces().entrySet().stream() + .filter(this::findAliveKafkaThread) + .collect(Collectors.toList()); if (leaks.isEmpty()) { return; } @@ -623,10 +620,9 @@ private void checkProducerLeak() throws InterruptedException { } private String format(Map.Entry leak) { - String stackTrace = - Arrays.stream(leak.getValue()) - .map(StackTraceElement::toString) - .collect(Collectors.joining("\n")); + String stackTrace = Arrays.stream(leak.getValue()) + .map(StackTraceElement::toString) + .collect(Collectors.joining("\n")); return leak.getKey().getName() + ":\n" + stackTrace; } @@ -689,6 +685,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } @Override - public void initializeState(FunctionInitializationContext context) throws Exception {} + public void initializeState(FunctionInitializationContext context) throws Exception { + } } } 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 1497c9bb9..2b0f41dc2 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 c9eceb982..80d17e487 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 @@ -90,24 +90,23 @@ public class KafkaWriterITCase { private MetricListener metricListener; private TriggerTimeService timeService; - private static final KafkaContainer KAFKA_CONTAINER = - createKafkaContainer(KAFKA, LOG) - .withEmbeddedZookeeper() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); + private static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KAFKA, LOG) + .withEmbeddedZookeeper() + .withNetwork(NETWORK) + .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", ""); @@ -115,27 +114,25 @@ public void setUp(TestInfo testInfo) { @ParameterizedTest @EnumSource(DeliveryGuarantee.class) - public void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter ignored = - createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { + void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { + try (final KafkaWriter ignored = createWriterWithConfiguration(getKafkaClientConfiguration(), + guarantee)) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME).isPresent()).isTrue(); } } @ParameterizedTest @EnumSource(DeliveryGuarantee.class) - public void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { + void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { assertKafkaMetricNotPresent(guarantee, "flink.disable-metrics", "true"); assertKafkaMetricNotPresent(guarantee, "register.producer.metrics", "false"); } @Test - public void testIncreasingRecordBasedCounters() throws Exception { + void testIncreasingRecordBasedCounters() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { + try (final KafkaWriter writer = createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { final Counter numBytesOut = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); final Counter numRecordsOut = metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); @@ -164,12 +161,10 @@ public void testIncreasingRecordBasedCounters() throws Exception { } @Test - public void testCurrentSendTimeMetric() throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE)) { - final Optional> currentSendTime = - metricListener.getGauge("currentSendTime"); + void testCurrentSendTimeMetric() throws Exception { + try (final KafkaWriter writer = createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE)) { + final Optional> currentSendTime = metricListener.getGauge("currentSendTime"); assertThat(currentSendTime.isPresent()).isTrue(); assertThat(currentSendTime.get().getValue()).isEqualTo(0L); IntStream.range(0, 100) @@ -195,9 +190,8 @@ void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final KafkaWriter writer = createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); @@ -220,9 +214,8 @@ void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final KafkaWriter writer = createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); @@ -244,14 +237,11 @@ void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = - new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + SinkInitContext sinkInitContext = new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); - final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + final KafkaWriter writer = createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); triggerProducerException(writer, properties); @@ -275,9 +265,8 @@ void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - final KafkaWriter writer = - createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final KafkaWriter writer = createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); @@ -296,8 +285,8 @@ private void triggerProducerException(KafkaWriter writer, Properties pr throws IOException { final String transactionalId = writer.getCurrentProducer().getTransactionalId(); - try (FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + try (FlinkKafkaInternalProducer producer = new FlinkKafkaInternalProducer<>(properties, + transactionalId)) { producer.initTransactions(); producer.beginTransaction(); producer.send(new ProducerRecord(topic, "1".getBytes())); @@ -308,14 +297,13 @@ 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( - getKafkaClientConfiguration(), - DeliveryGuarantee.AT_LEAST_ONCE, - createSinkWriterMetricGroup(), - meta -> metadataList.add(meta.toString()))) { + try (final KafkaWriter writer = createWriterWithConfiguration( + getKafkaClientConfiguration(), + DeliveryGuarantee.AT_LEAST_ONCE, + createSinkWriterMetricGroup(), + meta -> metadataList.add(meta.toString()))) { List expected = new ArrayList<>(); for (int i = 0; i < 100; i++) { writer.write(1, SINK_WRITER_CONTEXT); @@ -329,9 +317,8 @@ public void testMetadataPublisher() throws Exception { /** Test that producer is not accidentally recreated or pool is used. */ @Test void testLingeringTransaction() throws Exception { - final KafkaWriter failedWriter = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE); + final KafkaWriter failedWriter = createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE); // create two lingering transactions failedWriter.flush(false); @@ -341,9 +328,8 @@ void testLingeringTransaction() throws Exception { failedWriter.prepareCommit(); failedWriter.snapshotState(2); - try (final KafkaWriter recoveredWriter = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter recoveredWriter = createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { recoveredWriter.write(1, SINK_WRITER_CONTEXT); recoveredWriter.flush(false); @@ -355,8 +341,8 @@ void testLingeringTransaction() throws Exception { committable.getProducer().get().getObject().commitTransaction(); - List> records = - drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true); + List> records = drainAllRecordsFromTopic(topic, + getKafkaClientConfiguration(), true); assertThat(records).hasSize(1); } @@ -365,13 +351,10 @@ void testLingeringTransaction() throws Exception { /** Test that producer is not accidentally recreated or pool is used. */ @ParameterizedTest - @EnumSource( - value = DeliveryGuarantee.class, - names = "EXACTLY_ONCE", - mode = EnumSource.Mode.EXCLUDE) + @EnumSource(value = DeliveryGuarantee.class, names = "EXACTLY_ONCE", mode = EnumSource.Mode.EXCLUDE) void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { + try (final KafkaWriter writer = createWriterWithConfiguration(getKafkaClientConfiguration(), + guarantee)) { assertThat(writer.getProducerPool()).hasSize(0); FlinkKafkaInternalProducer firstProducer = writer.getCurrentProducer(); @@ -390,9 +373,8 @@ void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exce /** Test that producers are reused when committed. */ @Test void usePoolForTransactional() throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { assertThat(writer.getProducerPool()).hasSize(0); writer.write(1, SINK_WRITER_CONTEXT); @@ -403,8 +385,7 @@ void usePoolForTransactional() throws Exception { final KafkaCommittable committable = committables0.stream().findFirst().get(); assertThat(committable.getProducer().isPresent()).isTrue(); - FlinkKafkaInternalProducer firstProducer = - committable.getProducer().get().getObject(); + FlinkKafkaInternalProducer firstProducer = committable.getProducer().get().getObject(); assertThat(firstProducer != writer.getCurrentProducer()) .as("Expected different producer") .isTrue(); @@ -436,9 +417,8 @@ void usePoolForTransactional() throws Exception { */ @Test void prepareCommitForEmptyTransaction() throws Exception { - try (final KafkaWriter writer = - createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { assertThat(writer.getProducerPool()).hasSize(0); // no data written to current transaction @@ -447,8 +427,7 @@ void prepareCommitForEmptyTransaction() throws Exception { assertThat(emptyCommittables).hasSize(0); assertThat(writer.getProducerPool()).hasSize(1); - final FlinkKafkaInternalProducer recycledProducer = - writer.getProducerPool().pop(); + final FlinkKafkaInternalProducer recycledProducer = writer.getProducerPool().pop(); assertThat(recycledProducer.isInTransaction()).isFalse(); } } @@ -460,14 +439,14 @@ void prepareCommitForEmptyTransaction() throws Exception { @Test void testAbortOnClose() throws Exception { Properties properties = getKafkaClientConfiguration(); - try (final KafkaWriter writer = - createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = createWriterWithConfiguration(properties, + DeliveryGuarantee.EXACTLY_ONCE)) { writer.write(1, SINK_WRITER_CONTEXT); assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0); } - try (final KafkaWriter writer = - createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = createWriterWithConfiguration(properties, + DeliveryGuarantee.EXACTLY_ONCE)) { writer.write(2, SINK_WRITER_CONTEXT); writer.flush(false); Collection committables = writer.prepareCommit(); @@ -477,8 +456,8 @@ void testAbortOnClose() throws Exception { assertThat(committables).hasSize(1); final KafkaCommittable committable = committables.stream().findFirst().get(); String transactionalId = committable.getTransactionalId(); - try (FlinkKafkaInternalProducer producer = - new FlinkKafkaInternalProducer<>(properties, transactionalId)) { + try (FlinkKafkaInternalProducer producer = new FlinkKafkaInternalProducer<>(properties, + transactionalId)) { producer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); producer.commitTransaction(); } @@ -491,8 +470,7 @@ private void assertKafkaMetricNotPresent( DeliveryGuarantee guarantee, String configKey, String configValue) throws Exception { final Properties config = getKafkaClientConfiguration(); config.put(configKey, configValue); - try (final KafkaWriter ignored = - createWriterWithConfiguration(config, guarantee)) { + try (final KafkaWriter ignored = createWriterWithConfiguration(config, guarantee)) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isNotPresent(); } } @@ -516,34 +494,30 @@ private KafkaWriter createWriterWithConfiguration( SinkWriterMetricGroup sinkWriterMetricGroup, @Nullable Consumer metadataConsumer) throws IOException { - KafkaSink kafkaSink = - KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); - return (KafkaWriter) - kafkaSink.createWriter( - new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); + KafkaSink kafkaSink = KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); + return (KafkaWriter) kafkaSink.createWriter( + new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); } private KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) throws IOException { - KafkaSink kafkaSink = - KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); + KafkaSink kafkaSink = KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); return (KafkaWriter) kafkaSink.createWriter(sinkInitContext); } private SinkWriterMetricGroup createSinkWriterMetricGroup() { - DummyOperatorMetricGroup operatorMetricGroup = - new DummyOperatorMetricGroup(metricListener.getMetricGroup()); + DummyOperatorMetricGroup operatorMetricGroup = new DummyOperatorMetricGroup(metricListener.getMetricGroup()); return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); } @@ -562,7 +536,8 @@ private static class SinkInitContext extends TestSinkInitContext { private final SinkWriterMetricGroup metricGroup; private final ProcessingTimeService timeService; - @Nullable private final Consumer metadataConsumer; + @Nullable + private final Consumer metadataConsumer; SinkInitContext( SinkWriterMetricGroup metricGroup, @@ -609,8 +584,7 @@ public OptionalLong getRestoredCheckpointId() { } @Override - public SerializationSchema.InitializationContext - asSerializationSchemaInitializationContext() { + public SerializationSchema.InitializationContext asSerializationSchemaInitializationContext() { return null; } @@ -651,9 +625,8 @@ private static class DummyOperatorMetricGroup extends ProxyMetricGroup> registeredCallbacks = - new PriorityQueue<>(Comparator.comparingLong(o -> o.f0)); + private final PriorityQueue> registeredCallbacks = new PriorityQueue<>( + Comparator.comparingLong(o -> o.f0)); @Override public long getCurrentProcessingTime() { 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 38ef80d51..0d7c90bb7 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 deafb7d6d..a88181278 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; @@ -38,11 +37,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; @@ -79,7 +78,7 @@ */ @SuppressWarnings("serial") @RetryOnFailure(times = 3) -public abstract class KafkaTestBase extends TestLogger { +public abstract class KafkaTestBase { public static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class); @@ -96,18 +95,17 @@ public abstract class KafkaTestBase extends TestLogger { public static List kafkaClusters = new ArrayList<>(); - @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("-------------------------------------------------------------------------"); @@ -115,8 +113,8 @@ public static void prepare() throws Exception { startClusters(false, numKafkaClusters); } - @AfterClass - public static void shutDownServices() throws Exception { + @AfterAll + protected static void shutDownServices() throws Exception { LOG.info("-------------------------------------------------------------------------"); LOG.info(" Shut down KafkaTestBase "); @@ -153,9 +151,8 @@ public static void startClusters( KafkaTestEnvironment.Config environmentConfig, int numKafkaClusters) throws Exception { for (int i = 0; i < numKafkaClusters; i++) { startClusters(environmentConfig); - KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = - new KafkaClusterTestEnvMetadata( - i, kafkaServer, standardProps, brokerConnectionStrings, secureProps); + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = new KafkaClusterTestEnvMetadata( + i, kafkaServer, standardProps, brokerConnectionStrings, secureProps); kafkaClusters.add(kafkaClusterTestEnvMetadata); LOG.info("Created Kafka cluster with configuration: {}", kafkaClusterTestEnvMetadata); } @@ -183,9 +180,8 @@ public static void startClusters(KafkaTestEnvironment.Config environmentConfig) } public static KafkaTestEnvironment constructKafkaTestEnvironment() throws Exception { - Class clazz = - Class.forName( - "org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); + Class clazz = Class.forName( + "org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); return (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz); } @@ -241,8 +237,7 @@ public static void deleteTestTopic(String topic) { public static void produceToKafka( Collection> records, Class> keySerializerClass, - Class> - valueSerializerClass) + Class> valueSerializerClass) throws Throwable { Properties props = new Properties(); props.putAll(standardProps); @@ -252,14 +247,13 @@ public static void produceToKafka( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName()); AtomicReference sendingError = new AtomicReference<>(); - Callback callback = - (metadata, exception) -> { - if (exception != null) { - if (!sendingError.compareAndSet(null, exception)) { - sendingError.get().addSuppressed(exception); - } - } - }; + Callback callback = (metadata, exception) -> { + if (exception != null) { + if (!sendingError.compareAndSet(null, exception)) { + sendingError.get().addSuppressed(exception); + } + } + }; try (KafkaProducer producer = new KafkaProducer<>(props)) { for (ProducerRecord record : records) { producer.send(record, callback); @@ -301,8 +295,8 @@ public void assertAtLeastOnceForTopic( properties.put("heartbeat.interval.ms", "500"); // query kafka for new records ... - Collection> records = - kafkaServer.getAllRecordsFromTopic(properties, topic); + Collection> records = kafkaServer.getAllRecordsFromTopic(properties, + topic); for (ConsumerRecord record : records) { actualElements.add(record.value()); @@ -334,8 +328,8 @@ public void assertExactlyOnceForTopic( consumerProperties.put("isolation.level", "read_committed"); // query kafka for new records ... - Collection> records = - kafkaServer.getAllRecordsFromTopic(consumerProperties, topic); + Collection> records = kafkaServer.getAllRecordsFromTopic(consumerProperties, + topic); for (ConsumerRecord record : records) { actualElements.add(ByteBuffer.wrap(record.value()).getInt()); 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 cffe2d6c0..49e2246a7 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 120f1f047d9e0ca5301bd0d0723106ae5bc78bfb Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 09:41:56 +0800 Subject: [PATCH 02/15] [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 0d7c90bb7..277d38c9c 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 6853674b774e15f67257f78ed8556f4aca59444c Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 09:51:19 +0800 Subject: [PATCH 03/15] [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 fab61190b594ccc3677d65225532252e10ca82c8 Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 09:56:31 +0800 Subject: [PATCH 04/15] [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 f13778579b54250b38d171c662639ae579295d3c Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 10:00:51 +0800 Subject: [PATCH 05/15] [FLINK-25538][flink-connector-kafka] JUnit5 Migration --- .../apache/flink/streaming/connectors/kafka/KafkaTestBase.java | 3 --- 1 file changed, 3 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 a88181278..d06b899b5 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 @@ -95,9 +95,6 @@ public abstract class KafkaTestBase { public static List kafkaClusters = new ArrayList<>(); - @TempDir - public static File temporaryFolder; - public static Properties secureProps = new Properties(); // ------------------------------------------------------------------------ From d38dd8e9972e7d93ceb5ea86d5b4330e0dd3c79a Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 10:27:17 +0800 Subject: [PATCH 06/15] [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 000aa5abbf8f802cd83bcda2360b60433b90e95a Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Thu, 9 Nov 2023 10:34:37 +0800 Subject: [PATCH 07/15] [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 e2ccfd1e871c4fb603a41949791cf4e3d282d756 Mon Sep 17 00:00:00 2001 From: yu <13485876233> Date: Fri, 10 Nov 2023 09:52:50 +0800 Subject: [PATCH 08/15] [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; From f2eccc2d5248ca5c8dfbeec09f250e3cc8e14787 Mon Sep 17 00:00:00 2001 From: morazow Date: Wed, 19 Jun 2024 16:44:59 +0200 Subject: [PATCH 09/15] Fixed checkstyle findings --- .../kafka/sink/KafkaCommittableSerializerTest.java | 1 + .../KafkaRecordSerializationSchemaBuilderTest.java | 2 +- .../flink/connector/kafka/sink/KafkaSinkITCase.java | 10 ++++------ .../kafka/sink/KafkaTransactionLogITCase.java | 3 +-- .../connector/kafka/sink/TransactionIdFactoryTest.java | 3 +-- .../connectors/kafka/FlinkKafkaConsumerBaseTest.java | 1 - .../kafka/FlinkKafkaInternalProducerITCase.java | 2 +- .../connectors/kafka/KafkaShortRetentionTestBase.java | 2 +- .../streaming/connectors/kafka/KafkaTestBase.java | 4 ---- .../internals/AbstractPartitionDiscovererTest.java | 2 +- .../kafka/internals/KafkaTopicsDescriptorTest.java | 2 +- .../connectors/kafka/table/KafkaTableTestBase.java | 2 +- .../kafka/table/ReducingUpsertWriterTest.java | 2 +- .../connectors/kafka/table/UpsertKafkaTableITCase.java | 1 - 14 files changed, 14 insertions(+), 23 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 069afdb86..c77cf355a 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 @@ -18,6 +18,7 @@ package org.apache.flink.connector.kafka.sink; import org.apache.flink.util.TestLoggerExtension; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; 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 66c6c2bd6..c51aaf7a4 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.TestLoggerExtension; + import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.Configurable; import org.apache.kafka.common.header.Header; 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 d4d2ac444..132801194 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 @@ -59,30 +59,28 @@ 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.jupiter.api.AfterEach; import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.AfterEach; 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.jupiter.api.Test; 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; import org.testcontainers.containers.Network; +import org.testcontainers.junit.jupiter.Container; import org.testcontainers.utility.DockerImageName; import javax.annotation.Nullable; 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 2b0f41dc2..97e5a2ee4 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 @@ -18,9 +18,8 @@ package org.apache.flink.connector.kafka.sink; 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; 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 2054c420d..05d1277d5 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 @@ -17,9 +17,8 @@ package org.apache.flink.connector.kafka.sink; -import org.apache.flink.util.TestLogger; - import org.apache.flink.util.TestLoggerExtension; + import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; 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 c9555f9de..6a887e91a 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 @@ -65,7 +65,6 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; -import org.apache.flink.util.TestLogger; import org.apache.flink.util.function.SupplierWithException; import org.apache.flink.util.function.ThrowingRunnable; 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 14045427f..c90014242 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,8 +27,8 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Timeout; 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..b201a5c15 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,9 +36,9 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; -import org.testcontainers.junit.jupiter.Container; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.junit.jupiter.Container; import java.io.Serializable; import java.util.Properties; 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 d06b899b5..02bd442f7 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 @@ -29,7 +29,6 @@ import org.apache.flink.test.util.SuccessException; import org.apache.flink.testutils.junit.RetryOnFailure; import org.apache.flink.util.InstantiationUtil; -import org.apache.flink.util.TestLogger; import com.google.common.base.MoreObjects; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -39,9 +38,6 @@ import org.apache.kafka.clients.producer.ProducerRecord; 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; 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 4b72e7034..35bace6f2 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 @@ -19,9 +19,9 @@ package org.apache.flink.streaming.connectors.kafka.internals; import org.apache.flink.streaming.connectors.kafka.testutils.TestPartitionDiscoverer; - 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; 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 f4bbd8cd8..1fc998549 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 @@ -19,9 +19,9 @@ 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; import java.util.Collection; 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 49e2246a7..20773460b 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 @@ -37,11 +37,11 @@ import org.apache.kafka.common.serialization.StringDeserializer; 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; import org.testcontainers.containers.output.Slf4jLogConsumer; +import org.testcontainers.junit.jupiter.Container; import org.testcontainers.utility.DockerImageName; import java.time.Duration; 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 1186c7202..f83111a05 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 @@ -32,9 +32,9 @@ import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.runtime.typeutils.RowDataSerializer; - 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; 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 96e615623..57c85790f 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 @@ -23,7 +23,6 @@ import org.apache.flink.table.api.Table; 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; From 3be7a8796c66f071f26ed4fb346e3cba5c4287bc Mon Sep 17 00:00:00 2001 From: morazow Date: Wed, 19 Jun 2024 17:22:13 +0200 Subject: [PATCH 10/15] Revert formatting --- .../connector/kafka/sink/KafkaSinkITCase.java | 140 +++++++++--------- 1 file changed, 74 insertions(+), 66 deletions(-) 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 132801194..ad9c11929 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 @@ -32,7 +32,6 @@ import org.apache.flink.configuration.StateBackendOptions; import org.apache.flink.connector.base.DeliveryGuarantee; import org.apache.flink.connector.kafka.sink.testutils.KafkaSinkExternalContextFactory; -import org.apache.flink.connector.kafka.testutils.DockerImageVersions; import org.apache.flink.connector.kafka.testutils.KafkaUtil; import org.apache.flink.connector.testframe.environment.MiniClusterTestEnvironment; import org.apache.flink.connector.testframe.external.DefaultContainerizedExternalSystem; @@ -104,13 +103,13 @@ import java.util.stream.Collectors; import java.util.stream.LongStream; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; +import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.fail; /** Tests for using KafkaSink writing to a Kafka cluster. */ -@ExtendWith({ TestLoggerExtension.class }) +@ExtendWith({TestLoggerExtension.class}) class KafkaSinkITCase { private static final Logger LOG = LoggerFactory.getLogger(KafkaSinkITCase.class); @@ -127,16 +126,15 @@ class KafkaSinkITCase { private SharedReference lastCheckpointedRecord; @Container - public static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KAFKA, LOG) - .withEmbeddedZookeeper() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); + public static final KafkaContainer KAFKA_CONTAINER = + createKafkaContainer(KAFKA, LOG) + .withEmbeddedZookeeper() + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); - @RegisterExtension - public final SharedObjects sharedObjects = SharedObjects.create(); + @RegisterExtension public final SharedObjects sharedObjects = SharedObjects.create(); - @TempDir - public Path temp; + @TempDir public Path temp; @BeforeAll static void setupAdmin() { @@ -177,22 +175,24 @@ class IntegrationTests extends SinkTestSuiteBase { // Defines external system @TestExternalSystem - DefaultContainerizedExternalSystem kafka = DefaultContainerizedExternalSystem.builder() - .fromContainer( - new KafkaContainer( - DockerImageName.parse(DockerImageVersions.KAFKA))) - .build(); + DefaultContainerizedExternalSystem kafka = + DefaultContainerizedExternalSystem.builder() + .fromContainer( + new KafkaContainer( + DockerImageName.parse(DockerImageVersions.KAFKA))) + .build(); @SuppressWarnings("unused") @TestSemantics - CheckpointingMode[] semantics = new CheckpointingMode[] { - CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE - }; + CheckpointingMode[] semantics = + new CheckpointingMode[] { + CheckpointingMode.EXACTLY_ONCE, CheckpointingMode.AT_LEAST_ONCE + }; @SuppressWarnings("unused") @TestContext - KafkaSinkExternalContextFactory sinkContext = new KafkaSinkExternalContextFactory(kafka.getContainer(), - Collections.emptyList()); + KafkaSinkExternalContextFactory sinkContext = + new KafkaSinkExternalContextFactory(kafka.getContainer(), Collections.emptyList()); } @Test @@ -223,11 +223,12 @@ void testRecoveryWithExactlyOnceGuarantee() throws Exception { testRecoveryWithAssertion( DeliveryGuarantee.EXACTLY_ONCE, 1, - (records) -> assertThat(records) - .contains( - (LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new)))); + (records) -> + assertThat(records) + .contains( + (LongStream.range(1, lastCheckpointedRecord.get().get() + 1) + .boxed() + .toArray(Long[]::new)))); } @Test @@ -235,11 +236,12 @@ void testRecoveryWithExactlyOnceGuaranteeAndConcurrentCheckpoints() throws Excep testRecoveryWithAssertion( DeliveryGuarantee.EXACTLY_ONCE, 2, - (records) -> assertThat(records) - .contains( - LongStream.range(1, lastCheckpointedRecord.get().get() + 1) - .boxed() - .toArray(Long[]::new))); + (records) -> + assertThat(records) + .contains( + LongStream.range(1, lastCheckpointedRecord.get().get() + 1) + .boxed() + .toArray(Long[]::new))); } @Test @@ -270,7 +272,8 @@ void testAbortTransactionsOfPendingCheckpointsAfterFailure() throws Exception { failed.get().set(true); executeWithMapper( new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, "newPrefix"); - final List> collectedRecords = drainAllRecordsFromTopic(topic, true); + final List> collectedRecords = + drainAllRecordsFromTopic(topic, true); assertThat(deserializeValues(collectedRecords)) .contains( LongStream.range(1, lastCheckpointedRecord.get().get() + 1) @@ -296,7 +299,8 @@ void testAbortTransactionsAfterScaleInBeforeFirstCheckpoint() throws Exception { failed.get().set(true); executeWithMapper( new FailingCheckpointMapper(failed, lastCheckpointedRecord), config, null); - final List> collectedRecords = drainAllRecordsFromTopic(topic, true); + final List> collectedRecords = + drainAllRecordsFromTopic(topic, true); assertThat(deserializeValues(collectedRecords)) .contains( LongStream.range(1, lastCheckpointedRecord.get().get() + 1) @@ -314,14 +318,15 @@ private void executeWithMapper( env.setRestartStrategy(RestartStrategies.noRestart()); final DataStreamSource source = env.fromSequence(1, 10); final DataStream stream = source.map(mapper); - final KafkaSinkBuilder builder = new KafkaSinkBuilder() - .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) - .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) - .setRecordSerializer( - KafkaRecordSerializationSchema.builder() - .setTopic(topic) - .setValueSerializationSchema(new RecordSerializer()) - .build()); + final KafkaSinkBuilder builder = + new KafkaSinkBuilder() + .setDeliveryGuarantee(DeliveryGuarantee.EXACTLY_ONCE) + .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) + .setRecordSerializer( + KafkaRecordSerializationSchema.builder() + .setTopic(topic) + .setValueSerializationSchema(new RecordSerializer()) + .build()); if (transactionalIdPrefix == null) { transactionalIdPrefix = "kafka-sink"; } @@ -340,7 +345,8 @@ private void testRecoveryWithAssertion( env.enableCheckpointing(300L); env.getCheckpointConfig().setMaxConcurrentCheckpoints(maxConcurrentCheckpoints); DataStreamSource source = env.fromSequence(1, 10); - DataStream stream = source.map(new FailingCheckpointMapper(failed, lastCheckpointedRecord)); + DataStream stream = + source.map(new FailingCheckpointMapper(failed, lastCheckpointedRecord)); stream.sinkTo( new KafkaSinkBuilder() @@ -355,8 +361,8 @@ private void testRecoveryWithAssertion( .build()); env.execute(); - final List> collectedRecords = drainAllRecordsFromTopic(topic, - guarantee == DeliveryGuarantee.EXACTLY_ONCE); + final List> collectedRecords = + drainAllRecordsFromTopic(topic, guarantee == DeliveryGuarantee.EXACTLY_ONCE); recordsAssertion.accept(deserializeValues(collectedRecords)); checkProducerLeak(); } @@ -366,9 +372,10 @@ private void writeRecordsToKafka( throws Exception { final StreamExecutionEnvironment env = new LocalStreamEnvironment(); env.enableCheckpointing(100L); - final DataStream source = env.addSource( - new InfiniteIntegerSource( - emittedRecordsCount, emittedRecordsWithCheckpoint)); + final DataStream source = + env.addSource( + new InfiniteIntegerSource( + emittedRecordsCount, emittedRecordsWithCheckpoint)); source.sinkTo( new KafkaSinkBuilder() .setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers()) @@ -382,8 +389,9 @@ private void writeRecordsToKafka( .build()); env.execute(); - final List> collectedRecords = drainAllRecordsFromTopic( - topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); + final List> collectedRecords = + drainAllRecordsFromTopic( + topic, deliveryGuarantee == DeliveryGuarantee.EXACTLY_ONCE); final long recordsCount = expectedRecords.get().get(); assertThat(recordsCount).isEqualTo(collectedRecords.size()); assertThat(deserializeValues(collectedRecords)) @@ -418,9 +426,10 @@ private static Properties getKafkaClientConfiguration() { private void createTestTopic(String topic, int numPartitions, short replicationFactor) throws ExecutionException, InterruptedException, TimeoutException { - final CreateTopicsResult result = admin.createTopics( - Collections.singletonList( - new NewTopic(topic, numPartitions, replicationFactor))); + final CreateTopicsResult result = + admin.createTopics( + Collections.singletonList( + new NewTopic(topic, numPartitions, replicationFactor))); result.all().get(); } @@ -448,8 +457,8 @@ public byte[] serialize(Long element) { private static class FailAsyncCheckpointMapper implements MapFunction, CheckpointedFunction { - private static final ListStateDescriptor stateDescriptor = new ListStateDescriptor<>("test-state", - new SlowSerializer()); + private static final ListStateDescriptor stateDescriptor = + new ListStateDescriptor<>("test-state", new SlowSerializer()); private int failAfterCheckpoint; private ListState state; @@ -528,8 +537,7 @@ public Integer deserialize(Integer reuse, DataInputView source) throws IOExcepti } @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - } + public void copy(DataInputView source, DataOutputView target) throws IOException {} @Override public TypeSerializerSnapshot snapshotConfiguration() { @@ -593,16 +601,16 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - } + public void initializeState(FunctionInitializationContext context) throws Exception {} } private void checkProducerLeak() throws InterruptedException { List> leaks = null; for (int tries = 0; tries < 10; tries++) { - leaks = Thread.getAllStackTraces().entrySet().stream() - .filter(this::findAliveKafkaThread) - .collect(Collectors.toList()); + leaks = + Thread.getAllStackTraces().entrySet().stream() + .filter(this::findAliveKafkaThread) + .collect(Collectors.toList()); if (leaks.isEmpty()) { return; } @@ -618,9 +626,10 @@ private void checkProducerLeak() throws InterruptedException { } private String format(Map.Entry leak) { - String stackTrace = Arrays.stream(leak.getValue()) - .map(StackTraceElement::toString) - .collect(Collectors.joining("\n")); + String stackTrace = + Arrays.stream(leak.getValue()) + .map(StackTraceElement::toString) + .collect(Collectors.joining("\n")); return leak.getKey().getName() + ":\n" + stackTrace; } @@ -683,7 +692,6 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - } + public void initializeState(FunctionInitializationContext context) throws Exception {} } } From eff60b4f0edffbaa5c27d87dbf9e5d416aa4a854 Mon Sep 17 00:00:00 2001 From: morazow Date: Wed, 19 Jun 2024 17:25:02 +0200 Subject: [PATCH 11/15] Add missing import --- .../org/apache/flink/connector/kafka/sink/KafkaSinkITCase.java | 1 + 1 file changed, 1 insertion(+) 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 ad9c11929..9484cae7a 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 @@ -58,6 +58,7 @@ 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.TestLoggerExtension; import org.apache.kafka.clients.CommonClientConfigs; From 41dfd64605ac662cceac5cd5878a779b7c27abf8 Mon Sep 17 00:00:00 2001 From: morazow Date: Wed, 19 Jun 2024 18:19:40 +0200 Subject: [PATCH 12/15] Minor changes --- .../kafka/FlinkKafkaConsumerBaseTest.java | 39 ++++++++----------- 1 file changed, 17 insertions(+), 22 deletions(-) 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 6a887e91a..99a0932f2 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 @@ -109,18 +109,17 @@ class FlinkKafkaConsumerBaseTest { @Test @SuppressWarnings("unchecked") void testEitherWatermarkExtractor() { + final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer(); assertThatThrownBy( () -> - new DummyFlinkKafkaConsumer() - .assignTimestampsAndWatermarks( - (AssignerWithPeriodicWatermarks) null)) + consumer.assignTimestampsAndWatermarks( + (AssignerWithPeriodicWatermarks) null)) .isInstanceOf(NullPointerException.class); assertThatThrownBy( () -> - new DummyFlinkKafkaConsumer() - .assignTimestampsAndWatermarks( - (AssignerWithPunctuatedWatermarks) null)) + consumer.assignTimestampsAndWatermarks( + (AssignerWithPunctuatedWatermarks) null)) .isInstanceOf(NullPointerException.class); final AssignerWithPeriodicWatermarks periodicAssigner = @@ -161,7 +160,7 @@ void ignoreCheckpointWhenNotRunning() throws Exception { // acknowledgement of the checkpoint should also not result in any offset commits consumer.notifyCheckpointComplete(1L); assertThat(fetcher.getAndClearLastCommittedOffsets()).isNull(); - assertThat(fetcher.getCommitCount()).isEqualTo(0); + assertThat(fetcher.getCommitCount()).isZero(); } /** @@ -373,7 +372,7 @@ private void checkFilterRestoredPartitionsWithDisovered( @Test @SuppressWarnings("unchecked") - public void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { + void testSnapshotStateWithCommitOnCheckpointsEnabled() throws Exception { // -------------------------------------------------------------------- // prepare fake states @@ -430,7 +429,7 @@ public void go() throws Exception { assertThat(snapshot1).isEqualTo(state1); assertThat(consumer.getPendingOffsetsToCommit()).hasSize(1); - assertThat(consumer.getPendingOffsetsToCommit().get(138L)).isEqualTo(state1); + assertThat(consumer.getPendingOffsetsToCommit()).containsEntry(138L, state1); // checkpoint 2 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140)); @@ -445,7 +444,7 @@ public void go() throws Exception { assertThat(snapshot2).isEqualTo(state2); assertThat(consumer.getPendingOffsetsToCommit()).hasSize(2); - assertThat(consumer.getPendingOffsetsToCommit().get(140L)).isEqualTo(state2); + assertThat(consumer.getPendingOffsetsToCommit()).containsEntry(140L, state2); // ack checkpoint 1 consumer.notifyCheckpointComplete(138L); @@ -467,7 +466,7 @@ public void go() throws Exception { assertThat(snapshot3).isEqualTo(state3); assertThat(consumer.getPendingOffsetsToCommit()).hasSize(2); - assertThat(consumer.getPendingOffsetsToCommit().get(141L)).isEqualTo(state3); + assertThat(consumer.getPendingOffsetsToCommit()).containsEntry(141L, state3); // ack checkpoint 3, subsumes number 2 consumer.notifyCheckpointComplete(141L); @@ -486,7 +485,7 @@ public void go() throws Exception { @Test @SuppressWarnings("unchecked") - public void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception { + void testSnapshotStateWithCommitOnCheckpointsDisabled() throws Exception { // -------------------------------------------------------------------- // prepare fake states // -------------------------------------------------------------------- @@ -542,8 +541,7 @@ public void go() throws Exception { } assertThat(snapshot1).isEqualTo(state1); - assertThat(consumer.getPendingOffsetsToCommit().size()) - .isEqualTo(0); // pending offsets to commit should not be updated + assertThat(consumer.getPendingOffsetsToCommit()).isEmpty(); // pending offsets to commit should not be updated // checkpoint 2 consumer.snapshotState(new StateSnapshotContextSynchronousImpl(140, 140)); @@ -557,12 +555,11 @@ public void go() throws Exception { } assertThat(snapshot2).isEqualTo(state2); - assertThat(consumer.getPendingOffsetsToCommit().size()) - .isEqualTo(0); // pending offsets to commit should not be updated + assertThat(consumer.getPendingOffsetsToCommit()).isEmpty(); // pending offsets to commit should not be updated // ack checkpoint 1 consumer.notifyCheckpointComplete(138L); - assertThat(fetcher.getCommitCount()).isEqualTo(0); + assertThat(fetcher.getCommitCount()).isZero(); assertThat(fetcher.getAndClearLastCommittedOffsets()) .isNull(); // no offsets should be committed @@ -578,17 +575,16 @@ public void go() throws Exception { } assertThat(snapshot3).isEqualTo(state3); - assertThat(consumer.getPendingOffsetsToCommit().size()) - .isEqualTo(0); // pending offsets to commit should not be updated + assertThat(consumer.getPendingOffsetsToCommit()).isEmpty(); // pending offsets to commit should not be updated // ack checkpoint 3, subsumes number 2 consumer.notifyCheckpointComplete(141L); - assertThat(fetcher.getCommitCount()).isEqualTo(0); + assertThat(fetcher.getCommitCount()).isZero(); assertThat(fetcher.getAndClearLastCommittedOffsets()) .isNull(); // no offsets should be committed consumer.notifyCheckpointComplete(666); // invalid checkpoint - assertThat(fetcher.getCommitCount()).isEqualTo(0); + assertThat(fetcher.getCommitCount()).isZero(); assertThat(fetcher.getAndClearLastCommittedOffsets()) .isNull(); // no offsets should be committed @@ -1356,7 +1352,6 @@ public void addAll(List values) throws Exception { } } - @SuppressWarnings("unchecked") private static void setupConsumer( FlinkKafkaConsumerBase consumer, boolean isRestored, From d635f517a8ce4b349cbab646ad188b7f6d4e8b9b Mon Sep 17 00:00:00 2001 From: morazow Date: Thu, 20 Jun 2024 00:41:47 +0200 Subject: [PATCH 13/15] Updated tests --- .../kafka/sink/KafkaWriterITCase.java | 338 ++++++++++-------- .../sink/KafkaWriterStateSerializerTest.java | 5 +- .../sink/TransactionToAbortCheckerTest.java | 3 + .../enumerator/KafkaEnumeratorTest.java | 21 +- .../kafka/FlinkKafkaConsumerBaseTest.java | 74 ++-- .../FlinkKafkaInternalProducerITCase.java | 42 +-- .../kafka/FlinkKafkaProducerBaseTest.java | 52 +-- .../connectors/kafka/KafkaITCase.java | 97 +++-- .../connectors/kafka/KafkaTestBase.java | 3 + .../UpsertKafkaDynamicTableFactoryTest.java | 141 ++++---- 10 files changed, 384 insertions(+), 392 deletions(-) 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 80d17e487..f2043cfdb 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 @@ -27,11 +27,9 @@ import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; -import org.apache.flink.metrics.groups.OperatorMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; -import org.apache.flink.runtime.metrics.groups.ProxyMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.UserCodeClassLoader; @@ -60,6 +58,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Optional; @@ -70,30 +69,31 @@ import java.util.function.Consumer; import java.util.stream.IntStream; -import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; +import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; /** Tests for the standalone KafkaWriter. */ @ExtendWith(TestLoggerExtension.class) -public class KafkaWriterITCase { +class KafkaWriterITCase { private static final Logger LOG = LoggerFactory.getLogger(KafkaWriterITCase.class); private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka"; private static final Network NETWORK = Network.newNetwork(); private static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total"; private static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); - private static String topic; + private String topic; private MetricListener metricListener; private TriggerTimeService timeService; - private static final KafkaContainer KAFKA_CONTAINER = createKafkaContainer(KAFKA, LOG) - .withEmbeddedZookeeper() - .withNetwork(NETWORK) - .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); + private static final KafkaContainer KAFKA_CONTAINER = + createKafkaContainer(KAFKA, LOG) + .withEmbeddedZookeeper() + .withNetwork(NETWORK) + .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS); @BeforeAll static void beforeAll() { @@ -115,9 +115,9 @@ void setUp(TestInfo testInfo) { @ParameterizedTest @EnumSource(DeliveryGuarantee.class) void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter ignored = createWriterWithConfiguration(getKafkaClientConfiguration(), - guarantee)) { - assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME).isPresent()).isTrue(); + try (final KafkaWriter ignored = + createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { + assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isPresent(); } } @@ -130,43 +130,54 @@ void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { @Test void testIncreasingRecordBasedCounters() throws Exception { - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { + final OperatorIOMetricGroup operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); + final InternalSinkWriterMetricGroup metricGroup = + InternalSinkWriterMetricGroup.mock( + metricListener.getMetricGroup(), operatorIOMetricGroup); + try (final KafkaWriter writer = + createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { final Counter numBytesOut = metricGroup.getIOMetricGroup().getNumBytesOutCounter(); final Counter numRecordsOut = metricGroup.getIOMetricGroup().getNumRecordsOutCounter(); final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); final Counter numRecordsSendErrors = metricGroup.getNumRecordsSendErrorsCounter(); - assertThat(numBytesOut.getCount()).isEqualTo(0L); - assertThat(numRecordsOut.getCount()).isEqualTo(0); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); - assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); + assertThat(numBytesOut.getCount()).isZero(); + assertThat(numRecordsOut.getCount()).isZero(); + assertThat(numRecordsOutErrors.getCount()).isZero(); + assertThat(numRecordsSendErrors.getCount()).isZero(); // elements for which the serializer returns null should be silently skipped writer.write(null, SINK_WRITER_CONTEXT); timeService.trigger(); - assertThat(numBytesOut.getCount()).isEqualTo(0L); - assertThat(numRecordsOut.getCount()).isEqualTo(0); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); - assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); + assertThat(numBytesOut.getCount()).isZero(); + assertThat(numRecordsOut.getCount()).isZero(); + assertThat(numRecordsOutErrors.getCount()).isZero(); + assertThat(numRecordsSendErrors.getCount()).isZero(); // but elements for which a non-null producer record is returned should count writer.write(1, SINK_WRITER_CONTEXT); timeService.trigger(); assertThat(numRecordsOut.getCount()).isEqualTo(1); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0); - assertThat(numRecordsSendErrors.getCount()).isEqualTo(0); - assertThat(numBytesOut.getCount()).isGreaterThan(0L); + assertThat(numRecordsOutErrors.getCount()).isZero(); + assertThat(numRecordsSendErrors.getCount()).isZero(); + assertThat(numBytesOut.getCount()).isPositive(); } } @Test void testCurrentSendTimeMetric() throws Exception { - try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE)) { - final Optional> currentSendTime = metricListener.getGauge("currentSendTime"); - assertThat(currentSendTime.isPresent()).isTrue(); - assertThat(currentSendTime.get().getValue()).isEqualTo(0L); + final InternalSinkWriterMetricGroup metricGroup = + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()); + try (final KafkaWriter writer = + createWriterWithConfiguration( + getKafkaClientConfiguration(), + DeliveryGuarantee.AT_LEAST_ONCE, + metricGroup)) { + final Optional> currentSendTime = + metricListener.getGauge("currentSendTime"); + assertThat(currentSendTime).isPresent(); + assertThat(currentSendTime.get().getValue()).isZero(); IntStream.range(0, 100) .forEach( (run) -> { @@ -180,7 +191,7 @@ void testCurrentSendTimeMetric() throws Exception { throw new RuntimeException("Failed writing Kafka record."); } }); - assertThat(currentSendTime.get().getValue()).isGreaterThan(0L); + assertThat(currentSendTime.get().getValue()).isPositive(); } } @@ -188,12 +199,17 @@ void testCurrentSendTimeMetric() throws Exception { void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - - final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isZero(); triggerProducerException(writer, properties); @@ -212,12 +228,17 @@ void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - - final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isZero(); triggerProducerException(writer, properties); // to ensure that the exceptional send request has completed @@ -237,12 +258,17 @@ void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); - - final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isZero(); triggerProducerException(writer, properties); // to ensure that the exceptional send request has completed @@ -263,12 +289,17 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); - - final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); - final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); - assertThat(numRecordsOutErrors.getCount()).isEqualTo(0L); + SinkInitContext sinkInitContext = + new SinkInitContext( + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + timeService, + null); + final KafkaWriter writer = + createWriterWithConfiguration( + properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); + final Counter numRecordsOutErrors = + sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + assertThat(numRecordsOutErrors.getCount()).isZero(); triggerProducerException(writer, properties); // to ensure that the exceptional send request has completed @@ -285,8 +316,8 @@ private void triggerProducerException(KafkaWriter writer, Properties pr throws IOException { final String transactionalId = writer.getCurrentProducer().getTransactionalId(); - try (FlinkKafkaInternalProducer producer = new FlinkKafkaInternalProducer<>(properties, - transactionalId)) { + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(properties, transactionalId)) { producer.initTransactions(); producer.beginTransaction(); producer.send(new ProducerRecord(topic, "1".getBytes())); @@ -299,11 +330,12 @@ private void triggerProducerException(KafkaWriter writer, Properties pr @Test void testMetadataPublisher() throws Exception { List metadataList = new ArrayList<>(); - try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), - DeliveryGuarantee.AT_LEAST_ONCE, - createSinkWriterMetricGroup(), - meta -> metadataList.add(meta.toString()))) { + try (final KafkaWriter writer = + createWriterWithConfiguration( + getKafkaClientConfiguration(), + DeliveryGuarantee.AT_LEAST_ONCE, + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + meta -> metadataList.add(meta.toString()))) { List expected = new ArrayList<>(); for (int i = 0; i < 100; i++) { writer.write(1, SINK_WRITER_CONTEXT); @@ -317,8 +349,9 @@ void testMetadataPublisher() throws Exception { /** Test that producer is not accidentally recreated or pool is used. */ @Test void testLingeringTransaction() throws Exception { - final KafkaWriter failedWriter = createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE); + final KafkaWriter failedWriter = + createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE); // create two lingering transactions failedWriter.flush(false); @@ -328,8 +361,9 @@ void testLingeringTransaction() throws Exception { failedWriter.prepareCommit(); failedWriter.snapshotState(2); - try (final KafkaWriter recoveredWriter = createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter recoveredWriter = + createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { recoveredWriter.write(1, SINK_WRITER_CONTEXT); recoveredWriter.flush(false); @@ -337,12 +371,12 @@ void testLingeringTransaction() throws Exception { recoveredWriter.snapshotState(1); assertThat(committables).hasSize(1); final KafkaCommittable committable = committables.stream().findFirst().get(); - assertThat(committable.getProducer().isPresent()).isTrue(); + assertThat(committable.getProducer()).isPresent(); committable.getProducer().get().getObject().commitTransaction(); - List> records = drainAllRecordsFromTopic(topic, - getKafkaClientConfiguration(), true); + List> records = + drainAllRecordsFromTopic(topic, getKafkaClientConfiguration(), true); assertThat(records).hasSize(1); } @@ -351,31 +385,35 @@ void testLingeringTransaction() throws Exception { /** Test that producer is not accidentally recreated or pool is used. */ @ParameterizedTest - @EnumSource(value = DeliveryGuarantee.class, names = "EXACTLY_ONCE", mode = EnumSource.Mode.EXCLUDE) + @EnumSource( + value = DeliveryGuarantee.class, + names = "EXACTLY_ONCE", + mode = EnumSource.Mode.EXCLUDE) void useSameProducerForNonTransactional(DeliveryGuarantee guarantee) throws Exception { - try (final KafkaWriter writer = createWriterWithConfiguration(getKafkaClientConfiguration(), - guarantee)) { - assertThat(writer.getProducerPool()).hasSize(0); + try (final KafkaWriter writer = + createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { + assertThat(writer.getProducerPool()).isEmpty(); FlinkKafkaInternalProducer firstProducer = writer.getCurrentProducer(); writer.flush(false); Collection committables = writer.prepareCommit(); writer.snapshotState(0); - assertThat(committables).hasSize(0); + assertThat(committables).isEmpty(); - assertThat(writer.getCurrentProducer() == firstProducer) + assertThat(writer.getCurrentProducer()) .as("Expected same producer") - .isTrue(); - assertThat(writer.getProducerPool()).hasSize(0); + .isSameAs(firstProducer); + assertThat(writer.getProducerPool()).isEmpty(); } } /** Test that producers are reused when committed. */ @Test void usePoolForTransactional() throws Exception { - try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).hasSize(0); + try (final KafkaWriter writer = + createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + assertThat(writer.getProducerPool()).isEmpty(); writer.write(1, SINK_WRITER_CONTEXT); writer.flush(false); @@ -383,15 +421,16 @@ void usePoolForTransactional() throws Exception { writer.snapshotState(1); assertThat(committables0).hasSize(1); final KafkaCommittable committable = committables0.stream().findFirst().get(); - assertThat(committable.getProducer().isPresent()).isTrue(); + assertThat(committable.getProducer()).isPresent(); - FlinkKafkaInternalProducer firstProducer = committable.getProducer().get().getObject(); - assertThat(firstProducer != writer.getCurrentProducer()) + FlinkKafkaInternalProducer firstProducer = + committable.getProducer().get().getObject(); + assertThat(firstProducer) .as("Expected different producer") - .isTrue(); + .isNotSameAs(writer.getCurrentProducer()); // recycle first producer, KafkaCommitter would commit it and then return it - assertThat(writer.getProducerPool()).hasSize(0); + assertThat(writer.getProducerPool()).isEmpty(); firstProducer.commitTransaction(); committable.getProducer().get().close(); assertThat(writer.getProducerPool()).hasSize(1); @@ -402,11 +441,11 @@ void usePoolForTransactional() throws Exception { writer.snapshotState(2); assertThat(committables1).hasSize(1); final KafkaCommittable committable1 = committables1.stream().findFirst().get(); - assertThat(committable1.getProducer().isPresent()).isTrue(); + assertThat(committable1.getProducer()).isPresent(); - assertThat(firstProducer == writer.getCurrentProducer()) + assertThat(firstProducer) .as("Expected recycled producer") - .isTrue(); + .isSameAs(writer.getCurrentProducer()); } } @@ -417,17 +456,19 @@ void usePoolForTransactional() throws Exception { */ @Test void prepareCommitForEmptyTransaction() throws Exception { - try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).hasSize(0); + try (final KafkaWriter writer = + createWriterWithConfiguration( + getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { + assertThat(writer.getProducerPool()).isEmpty(); // no data written to current transaction writer.flush(false); Collection emptyCommittables = writer.prepareCommit(); - assertThat(emptyCommittables).hasSize(0); + assertThat(emptyCommittables).isEmpty(); assertThat(writer.getProducerPool()).hasSize(1); - final FlinkKafkaInternalProducer recycledProducer = writer.getProducerPool().pop(); + final FlinkKafkaInternalProducer recycledProducer = + writer.getProducerPool().pop(); assertThat(recycledProducer.isInTransaction()).isFalse(); } } @@ -439,14 +480,14 @@ void prepareCommitForEmptyTransaction() throws Exception { @Test void testAbortOnClose() throws Exception { Properties properties = getKafkaClientConfiguration(); - try (final KafkaWriter writer = createWriterWithConfiguration(properties, - DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = + createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { writer.write(1, SINK_WRITER_CONTEXT); - assertThat(drainAllRecordsFromTopic(topic, properties, true)).hasSize(0); + assertThat(drainAllRecordsFromTopic(topic, properties, true)).isEmpty(); } - try (final KafkaWriter writer = createWriterWithConfiguration(properties, - DeliveryGuarantee.EXACTLY_ONCE)) { + try (final KafkaWriter writer = + createWriterWithConfiguration(properties, DeliveryGuarantee.EXACTLY_ONCE)) { writer.write(2, SINK_WRITER_CONTEXT); writer.flush(false); Collection committables = writer.prepareCommit(); @@ -456,8 +497,8 @@ void testAbortOnClose() throws Exception { assertThat(committables).hasSize(1); final KafkaCommittable committable = committables.stream().findFirst().get(); String transactionalId = committable.getTransactionalId(); - try (FlinkKafkaInternalProducer producer = new FlinkKafkaInternalProducer<>(properties, - transactionalId)) { + try (FlinkKafkaInternalProducer producer = + new FlinkKafkaInternalProducer<>(properties, transactionalId)) { producer.resumeTransaction(committable.getProducerId(), committable.getEpoch()); producer.commitTransaction(); } @@ -470,21 +511,24 @@ private void assertKafkaMetricNotPresent( DeliveryGuarantee guarantee, String configKey, String configValue) throws Exception { final Properties config = getKafkaClientConfiguration(); config.put(configKey, configValue); - try (final KafkaWriter ignored = createWriterWithConfiguration(config, guarantee)) { + try (final KafkaWriter ignored = + createWriterWithConfiguration(config, guarantee)) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isNotPresent(); } } private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee) throws IOException { - return createWriterWithConfiguration(config, guarantee, createSinkWriterMetricGroup()); + Properties config, DeliveryGuarantee guarantee) { + return createWriterWithConfiguration( + config, + guarantee, + InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); } private KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup) - throws IOException { + SinkWriterMetricGroup sinkWriterMetricGroup) { return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null); } @@ -492,33 +536,27 @@ private KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee, SinkWriterMetricGroup sinkWriterMetricGroup, - @Nullable Consumer metadataConsumer) - throws IOException { - KafkaSink kafkaSink = KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); - return (KafkaWriter) kafkaSink.createWriter( - new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); + @Nullable Consumer metadataConsumer) { + return new KafkaWriter<>( + guarantee, + config, + "test-prefix", + new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer), + new DummyRecordSerializer(), + new DummySchemaContext(), + Collections.emptyList()); } private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) - throws IOException { - KafkaSink kafkaSink = KafkaSink.builder() - .setKafkaProducerConfig(config) - .setDeliveryGuarantee(guarantee) - .setTransactionalIdPrefix("test-prefix") - .setRecordSerializer(new DummyRecordSerializer()) - .build(); - return (KafkaWriter) kafkaSink.createWriter(sinkInitContext); - } - - private SinkWriterMetricGroup createSinkWriterMetricGroup() { - DummyOperatorMetricGroup operatorMetricGroup = new DummyOperatorMetricGroup(metricListener.getMetricGroup()); - return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); + Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) { + return new KafkaWriter<>( + guarantee, + config, + "test-prefix", + sinkInitContext, + new DummyRecordSerializer(), + new DummySchemaContext(), + Collections.emptyList()); } private static Properties getKafkaClientConfiguration() { @@ -536,8 +574,7 @@ private static class SinkInitContext extends TestSinkInitContext { private final SinkWriterMetricGroup metricGroup; private final ProcessingTimeService timeService; - @Nullable - private final Consumer metadataConsumer; + @Nullable private final Consumer metadataConsumer; SinkInitContext( SinkWriterMetricGroup metricGroup, @@ -584,7 +621,8 @@ public OptionalLong getRestoredCheckpointId() { } @Override - public SerializationSchema.InitializationContext asSerializationSchemaInitializationContext() { + public SerializationSchema.InitializationContext + asSerializationSchemaInitializationContext() { return null; } @@ -594,7 +632,7 @@ public Optional> metadataConsumer() { } } - private static class DummyRecordSerializer implements KafkaRecordSerializationSchema { + private class DummyRecordSerializer implements KafkaRecordSerializationSchema { @Override public ProducerRecord serialize( Integer element, KafkaSinkContext context, Long timestamp) { @@ -606,39 +644,35 @@ public ProducerRecord serialize( } } - private static class DummySinkWriterContext implements SinkWriter.Context { + private static class DummySchemaContext implements SerializationSchema.InitializationContext { + @Override - public long currentWatermark() { - return 0; + public MetricGroup getMetricGroup() { + throw new UnsupportedOperationException("Not implemented."); } @Override - public Long timestamp() { - return null; + public UserCodeClassLoader getUserCodeClassLoader() { + throw new UnsupportedOperationException("Not implemented."); } } - private static class DummyOperatorMetricGroup extends ProxyMetricGroup - implements OperatorMetricGroup { - - private final OperatorIOMetricGroup operatorIOMetricGroup; - - public DummyOperatorMetricGroup(MetricGroup parentMetricGroup) { - super(parentMetricGroup); - this.operatorIOMetricGroup = UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup() - .getIOMetricGroup(); + private static class DummySinkWriterContext implements SinkWriter.Context { + @Override + public long currentWatermark() { + return 0; } @Override - public OperatorIOMetricGroup getIOMetricGroup() { - return operatorIOMetricGroup; + public Long timestamp() { + return null; } } private static class TriggerTimeService implements ProcessingTimeService { - private final PriorityQueue> registeredCallbacks = new PriorityQueue<>( - Comparator.comparingLong(o -> o.f0)); + private final PriorityQueue> registeredCallbacks = + new PriorityQueue<>(Comparator.comparingLong(o -> o.f0)); @Override public long getCurrentProcessingTime() { 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 f1987861e..d8c5549b1 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,16 +17,19 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.io.IOException; import static org.assertj.core.api.Assertions.assertThat; /** - * Tests for serializing and deserialzing {@link KafkaWriterState} with {@link + * Tests for serializing and deserializing {@link KafkaWriterState} with {@link * KafkaWriterStateSerializer}. */ +@ExtendWith(TestLoggerExtension.class) class KafkaWriterStateSerializerTest { private static final KafkaWriterStateSerializer SERIALIZER = new KafkaWriterStateSerializer(); 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 a4c1b64a1..ccf00f2c1 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,7 +17,9 @@ package org.apache.flink.connector.kafka.sink; +import org.apache.flink.util.TestLoggerExtension; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Collections; import java.util.HashMap; @@ -27,6 +29,7 @@ import static org.assertj.core.api.Assertions.assertThat; /** Tests for {@link TransactionsToAbortChecker}. */ +@ExtendWith(TestLoggerExtension.class) class TransactionToAbortCheckerTest { public static final String ABORT = "abort"; 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 0681b6385..4d7eb3282 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 @@ -51,6 +51,7 @@ import java.util.Properties; import java.util.Set; import java.util.StringJoiner; +import java.util.concurrent.TimeUnit; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -95,7 +96,7 @@ void testStartWithDiscoverPartitionsOnce() throws Exception { KafkaSourceEnumerator enumerator = createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign + // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. enumerator.start(); assertThat(context.getPeriodicCallables()).isEmpty(); @@ -116,7 +117,7 @@ void testStartWithPeriodicPartitionDiscovery() throws Exception { KafkaSourceEnumerator enumerator = createEnumerator(context, ENABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign + // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. enumerator.start(); assertThat(context.getOneTimeCallables()).isEmpty(); @@ -137,7 +138,7 @@ void testDiscoverPartitionsTriggersAssignments() throws Throwable { KafkaSourceEnumerator enumerator = createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign + // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. enumerator.start(); @@ -162,7 +163,7 @@ void testReaderRegistrationTriggersAssignments() throws Throwable { KafkaSourceEnumerator enumerator = createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign + // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. enumerator.start(); runOneTimePartitionDiscovery(context); @@ -185,7 +186,7 @@ void testRunWithDiscoverPartitionsOnceToCheckNoMoreSplit() throws Throwable { KafkaSourceEnumerator enumerator = createEnumerator(context, DISABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign + // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. enumerator.start(); assertThat(context.getOneTimeCallables()) @@ -209,7 +210,7 @@ void testRunWithPeriodicPartitionDiscoveryOnceToCheckNoMoreSplit() throws Throwa KafkaSourceEnumerator enumerator = createEnumerator(context, ENABLE_PERIODIC_PARTITION_DISCOVERY)) { - // Start the enumerator and it should schedule a one time task to discover and assign + // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. enumerator.start(); assertThat(context.getOneTimeCallables()).isEmpty(); @@ -251,7 +252,7 @@ void testRunWithDiscoverPartitionsOnceWithZeroMsToCheckNoMoreSplit() throws Thro } @Test - @Timeout(30L) + @Timeout(value = 30L, unit = TimeUnit.SECONDS) void testDiscoverPartitionsPeriodically() throws Throwable { try (MockSplitEnumeratorContext context = new MockSplitEnumeratorContext<>(NUM_SUBTASKS); @@ -485,7 +486,7 @@ void testPartitionChangeChecking() throws Throwable { final KafkaSourceEnumerator.PartitionChange partitionChange = enumerator.getPartitionChange(fetchedPartitions); - // Since enumerator never met DYNAMIC_TOPIC_NAME-0, it should be mark as a new partition + // Since enumerator never met DYNAMIC_TOPIC_NAME-0, it should be marked as a new partition Set expectedNewPartitions = Collections.singleton(newPartition); // All existing topics are not in the fetchedPartitions, so they should be marked as @@ -534,7 +535,7 @@ private void startEnumeratorAndRegisterReaders( MockSplitEnumeratorContext context, KafkaSourceEnumerator enumerator) throws Throwable { - // Start the enumerator and it should schedule a one time task to discover and assign + // Start the enumerator, and it should schedule a one time task to discover and assign // partitions. enumerator.start(); @@ -679,7 +680,7 @@ private void verifyAssignments( Set expectedAssignmentsForReader = expectedAssignments.get(reader); assertThat(expectedAssignmentsForReader).isNotNull(); - assertThat(splits.size()).isEqualTo(expectedAssignmentsForReader.size()); + assertThat(splits).hasSameSizeAs(expectedAssignmentsForReader); for (KafkaPartitionSplit split : splits) { assertThat(expectedAssignmentsForReader) .contains(split.getTopicPartition()); 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 99a0932f2..60cc00495 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 @@ -65,10 +65,12 @@ import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; import org.apache.flink.util.SerializedValue; +import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.function.SupplierWithException; import org.apache.flink.util.function.ThrowingRunnable; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import javax.annotation.Nonnull; @@ -101,6 +103,7 @@ import static org.mockito.Mockito.mock; /** Tests for the {@link FlinkKafkaConsumerBase}. */ +@ExtendWith(TestLoggerExtension.class) class FlinkKafkaConsumerBaseTest { private static final int maxParallelism = Short.MAX_VALUE / 2; @@ -141,7 +144,6 @@ void testEitherWatermarkExtractor() { /** Tests that no checkpoints happen when the fetcher is not running. */ @Test void ignoreCheckpointWhenNotRunning() throws Exception { - @SuppressWarnings("unchecked") final MockFetcher fetcher = new MockFetcher<>(); final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>( @@ -169,7 +171,6 @@ void ignoreCheckpointWhenNotRunning() throws Exception { */ @Test void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { - @SuppressWarnings("unchecked") final FlinkKafkaConsumerBase consumer = new DummyFlinkKafkaConsumer<>(); final TestingListState> restoredListState = @@ -202,7 +203,6 @@ void checkRestoredCheckpointWhenFetcherNotReady() throws Exception { @Test void testConfigureOnCheckpointsCommitMode() throws Exception { - @SuppressWarnings("unchecked") // auto-commit enabled; this should be ignored in this case final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); @@ -215,7 +215,6 @@ void testConfigureOnCheckpointsCommitMode() throws Exception { @Test void testConfigureAutoCommitMode() throws Exception { - @SuppressWarnings("unchecked") final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); setupConsumer(consumer); @@ -225,7 +224,6 @@ void testConfigureAutoCommitMode() throws Exception { @Test void testConfigureDisableOffsetCommitWithCheckpointing() throws Exception { - @SuppressWarnings("unchecked") // auto-commit enabled; this should be ignored in this case final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(true); consumer.setCommitOffsetsOnCheckpoints( @@ -240,7 +238,6 @@ void testConfigureDisableOffsetCommitWithCheckpointing() throws Exception { @Test void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Exception { - @SuppressWarnings("unchecked") final DummyFlinkKafkaConsumer consumer = new DummyFlinkKafkaConsumer<>(false); setupConsumer(consumer); @@ -253,8 +250,8 @@ void testConfigureDisableOffsetCommitWithoutCheckpointing() throws Exception { * (filterRestoredPartitionsWithDiscovered is active) */ @Test - void testSetFilterRestoredParitionsNoChange() throws Exception { - checkFilterRestoredPartitionsWithDisovered( + void testSetFilterRestoredPartitionsNoChange() throws Exception { + checkFilterRestoredPartitionsWithDiscovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -266,8 +263,8 @@ void testSetFilterRestoredParitionsNoChange() throws Exception { * in restored partitions. (filterRestoredPartitionsWithDiscovered is active) */ @Test - void testSetFilterRestoredParitionsWithRemovedTopic() throws Exception { - checkFilterRestoredPartitionsWithDisovered( + void testSetFilterRestoredPartitionsWithRemovedTopic() throws Exception { + checkFilterRestoredPartitionsWithDiscovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1"}), Arrays.asList(new String[] {"kafka_topic_1"}), @@ -279,8 +276,8 @@ void testSetFilterRestoredParitionsWithRemovedTopic() throws Exception { * (filterRestoredPartitionsWithDiscovered is active) */ @Test - void testSetFilterRestoredParitionsWithAddedTopic() throws Exception { - checkFilterRestoredPartitionsWithDisovered( + void testSetFilterRestoredPartitionsWithAddedTopic() throws Exception { + checkFilterRestoredPartitionsWithDiscovered( Arrays.asList(new String[] {"kafka_topic_1"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -292,8 +289,8 @@ void testSetFilterRestoredParitionsWithAddedTopic() throws Exception { * (filterRestoredPartitionsWithDiscovered is disabled) */ @Test - void testDisableFilterRestoredParitionsNoChange() throws Exception { - checkFilterRestoredPartitionsWithDisovered( + void testDisableFilterRestoredPartitionsNoChange() throws Exception { + checkFilterRestoredPartitionsWithDiscovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -305,8 +302,8 @@ void testDisableFilterRestoredParitionsNoChange() throws Exception { * still in restored partitions. (filterRestoredPartitionsWithDiscovered is disabled) */ @Test - void testDisableFilterRestoredParitionsWithRemovedTopic() throws Exception { - checkFilterRestoredPartitionsWithDisovered( + void testDisableFilterRestoredPartitionsWithRemovedTopic() throws Exception { + checkFilterRestoredPartitionsWithDiscovered( Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), @@ -318,15 +315,15 @@ void testDisableFilterRestoredParitionsWithRemovedTopic() throws Exception { * (filterRestoredPartitionsWithDiscovered is disabled) */ @Test - void testDisableFilterRestoredParitionsWithAddedTopic() throws Exception { - checkFilterRestoredPartitionsWithDisovered( + void testDisableFilterRestoredPartitionsWithAddedTopic() throws Exception { + checkFilterRestoredPartitionsWithDiscovered( Arrays.asList(new String[] {"kafka_topic_1"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), Arrays.asList(new String[] {"kafka_topic_1", "kafka_topic_2"}), true); } - private void checkFilterRestoredPartitionsWithDisovered( + private void checkFilterRestoredPartitionsWithDiscovered( List restoredKafkaTopics, List initKafkaTopics, List expectedSubscribedPartitions, @@ -940,7 +937,7 @@ public FailingPartitionDiscoverer(RuntimeException failureCause) { } @Override - protected void initializeConnections() throws Exception { + protected void initializeConnections() { closed = false; } @@ -948,7 +945,7 @@ protected void initializeConnections() throws Exception { protected void wakeupConnections() {} @Override - protected void closeConnections() throws Exception { + protected void closeConnections() { closed = true; } @@ -1082,8 +1079,7 @@ public void cancel() { @Override protected void doCommitInternalOffsetsToKafka( - Map offsets, @Nonnull KafkaCommitCallback commitCallback) - throws Exception {} + Map offsets, @Nonnull KafkaCommitCallback commitCallback) {} @Override protected KPH createKafkaPartitionHandle(KafkaTopicPartition partition) { @@ -1099,11 +1095,10 @@ protected KPH createKafkaPartitionHandle(KafkaTopicPartition partition) { private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase { private static final long serialVersionUID = 1L; - private SupplierWithException, Exception> testFetcherSupplier; - private AbstractPartitionDiscoverer testPartitionDiscoverer; - private boolean isAutoCommitEnabled; + private final SupplierWithException, Exception> testFetcherSupplier; + private final AbstractPartitionDiscoverer testPartitionDiscoverer; + private final boolean isAutoCommitEnabled; - @SuppressWarnings("unchecked") DummyFlinkKafkaConsumer() { this(false); } @@ -1144,7 +1139,6 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase) mock(KeyedDeserializationSchema.class)); } - @SuppressWarnings("unchecked") DummyFlinkKafkaConsumer( SupplierWithException, Exception> abstractFetcherSupplier, AbstractPartitionDiscoverer abstractPartitionDiscoverer, @@ -1156,7 +1150,6 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase testFetcher, AbstractPartitionDiscoverer testPartitionDiscoverer, @@ -1168,7 +1161,6 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase testFetcher, AbstractPartitionDiscoverer testPartitionDiscoverer, @@ -1196,7 +1188,6 @@ private static class DummyFlinkKafkaConsumer extends FlinkKafkaConsumerBase) mock(KeyedDeserializationSchema.class)); } - @SuppressWarnings("unchecked") DummyFlinkKafkaConsumer( SupplierWithException, Exception> testFetcherSupplier, AbstractPartitionDiscoverer testPartitionDiscoverer, @@ -1316,12 +1307,12 @@ public void clear() { } @Override - public Iterable get() throws Exception { + public Iterable get() { return list; } @Override - public void add(T value) throws Exception { + public void add(T value) { Preconditions.checkNotNull(value, "You cannot add null to a ListState."); list.add(value); } @@ -1335,14 +1326,13 @@ boolean isClearCalled() { } @Override - public void update(List values) throws Exception { + public void update(List values) { clear(); - addAll(values); } @Override - public void addAll(List values) throws Exception { + public void addAll(List values) { if (values != null) { values.forEach( v -> Preconditions.checkNotNull(v, "You cannot add null to a ListState.")); @@ -1400,8 +1390,7 @@ private MockFetcher(HashMap... stateSnapshotsToReturn @Override protected void doCommitInternalOffsetsToKafka( - Map offsets, @Nonnull KafkaCommitCallback commitCallback) - throws Exception { + Map offsets, @Nonnull KafkaCommitCallback commitCallback) { this.lastCommittedOffsets = offsets; this.commitCount++; commitCallback.onSuccess(); @@ -1454,20 +1443,17 @@ private MockOperatorStateStore(ListState restoredUnionListState) { @Override @SuppressWarnings("unchecked") - public ListState getUnionListState(ListStateDescriptor stateDescriptor) - throws Exception { + public ListState getUnionListState(ListStateDescriptor stateDescriptor) { return (ListState) mockRestoredUnionListState; } @Override - public BroadcastState getBroadcastState( - MapStateDescriptor stateDescriptor) throws Exception { + public BroadcastState getBroadcastState(MapStateDescriptor stateDescriptor) { throw new UnsupportedOperationException(); } @Override - public ListState getListState(ListStateDescriptor stateDescriptor) - throws Exception { + public ListState getListState(ListStateDescriptor stateDescriptor) { throw new UnsupportedOperationException(); } 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 c90014242..feb1fb876 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 @@ -84,8 +84,8 @@ void before() { } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testHappyPath() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testHappyPath() { String topicName = "flink-kafka-producer-happy-path"; Producer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); @@ -106,8 +106,8 @@ public void testHappyPath() throws Exception { } @Test - @Timeout(30L) - public void testResumeTransaction() throws Exception { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testResumeTransaction() { String topicName = "flink-kafka-producer-resume-transaction"; FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); @@ -135,7 +135,7 @@ public void testResumeTransaction() throws Exception { assertRecord(topicName, "42", "42"); // this shouldn't throw - in case of network split, old producer might attempt to commit - // it's transaction + // its transaction kafkaProducer.commitTransaction(); // this shouldn't fail also, for same reason as above @@ -153,8 +153,8 @@ public void testResumeTransaction() throws Exception { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testPartitionsForAfterClosed() { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testPartitionsForAfterClosed() { FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); kafkaProducer.close(Duration.ofSeconds(5)); @@ -163,8 +163,8 @@ public void testPartitionsForAfterClosed() { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testInitTransactionsAfterClosed() { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testInitTransactionsAfterClosed() { FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); kafkaProducer.close(Duration.ofSeconds(5)); @@ -173,8 +173,8 @@ public void testInitTransactionsAfterClosed() { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testBeginTransactionAfterClosed() { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testBeginTransactionAfterClosed() { FlinkKafkaInternalProducer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); kafkaProducer.initTransactions(); @@ -184,8 +184,8 @@ public void testBeginTransactionAfterClosed() { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testCommitTransactionAfterClosed() { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testCommitTransactionAfterClosed() { String topicName = "testCommitTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); assertThatThrownBy(kafkaProducer::commitTransaction) @@ -193,8 +193,8 @@ public void testCommitTransactionAfterClosed() { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testResumeTransactionAfterClosed() { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testResumeTransactionAfterClosed() { String topicName = "testAbortTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); assertThatThrownBy(() -> kafkaProducer.resumeTransaction(0L, (short) 1)) @@ -202,8 +202,8 @@ public void testResumeTransactionAfterClosed() { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testAbortTransactionAfterClosed() { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testAbortTransactionAfterClosed() { String topicName = "testAbortTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); kafkaProducer.abortTransaction(); @@ -212,8 +212,8 @@ public void testAbortTransactionAfterClosed() { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testFlushAfterClosed() { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testFlushAfterClosed() { String topicName = "testCommitTransactionAfterClosed"; FlinkKafkaInternalProducer kafkaProducer = getClosedProducer(topicName); assertThatThrownBy(kafkaProducer::flush) @@ -221,8 +221,8 @@ public void testFlushAfterClosed() { } @Test - @Timeout(value = 30, unit = TimeUnit.SECONDS) - public void testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator() throws Exception { + @Timeout(value = 30L, unit = TimeUnit.SECONDS) + void testProducerWhenCommitEmptyPartitionsToOutdatedTxnCoordinator() throws Exception { String topic = "flink-kafka-producer-txn-coordinator-changed-" + UUID.randomUUID(); createTestTopic(topic, 1, 1); Producer kafkaProducer = new FlinkKafkaInternalProducer<>(extraProperties); 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 5af52ec54..a269471b9 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 @@ -52,7 +52,6 @@ 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; import static org.mockito.Mockito.mock; @@ -65,7 +64,7 @@ class FlinkKafkaProducerBaseTest { /** Tests that the constructor eagerly checks bootstrap servers are set in config. */ @Test - public void testInstantiationFailsWhenBootstrapServersMissing() throws Exception { + void testInstantiationFailsWhenBootstrapServersMissing() throws Exception { // no bootstrap servers set in props Properties props = new Properties(); // should throw IllegalArgumentException @@ -169,17 +168,9 @@ void testAsyncErrorRethrownOnInvoke() throws Throwable { .get(0) .onCompletion(null, new Exception("artificial async exception")); - try { - testHarness.processElement(new StreamRecord<>("msg-2")); - } catch (Exception e) { - // the next invoke should rethrow the async exception - assertThat(e.getCause().getMessage()).contains("artificial async exception"); - - // test succeeded - return; - } - - fail("unknown failure"); + // the next invoke should rethrow the async exception + assertThatThrownBy(() -> testHarness.processElement(new StreamRecord<>("msg-2"))) + .hasStackTraceContaining("artificial async exception"); } /** @@ -206,17 +197,9 @@ void testAsyncErrorRethrownOnCheckpoint() throws Throwable { .get(0) .onCompletion(null, new Exception("artificial async exception")); - try { - testHarness.snapshot(123L, 123L); - } catch (Exception e) { - // the next invoke should rethrow the async exception - assertThat(e.getCause().getMessage()).contains("artificial async exception"); - - // test succeeded - return; - } - - fail("unknown failure"); + // the next invoke should rethrow the async exception + assertThatThrownBy(() -> testHarness.snapshot(123L, 123L)) + .hasStackTraceContaining("artificial async exception"); } /** @@ -229,7 +212,7 @@ void testAsyncErrorRethrownOnCheckpoint() throws Throwable { */ @SuppressWarnings("unchecked") @Test - @Timeout(value = 5, unit = TimeUnit.SECONDS) + @Timeout(value = 5L, unit = TimeUnit.SECONDS) void testAsyncErrorRethrownOnCheckpointAfterFlush() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( @@ -271,18 +254,9 @@ public void go() throws Exception { .onCompletion(null, new Exception("artificial async failure for 2nd message")); producer.getPendingCallbacks().get(2).onCompletion(null, null); - try { - snapshotThread.sync(); - } catch (Exception e) { - // the snapshot should have failed with the async exception - assertThat(e.getCause().getMessage()) - .contains("artificial async failure for 2nd message"); - - // test succeeded - return; - } - - fail("unknown failure"); + // the snapshot should have failed with the async exception + assertThatThrownBy(() -> snapshotThread.sync()) + .hasStackTraceContaining("artificial async failure for 2nd message"); } /** @@ -291,7 +265,7 @@ public void go() throws Exception { */ @SuppressWarnings("unchecked") @Test - @Timeout(value = 10, unit = TimeUnit.SECONDS) + @Timeout(value = 10L, unit = TimeUnit.SECONDS) void testAtLeastOnceProducer() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( @@ -365,7 +339,7 @@ public void go() throws Exception { */ @SuppressWarnings("unchecked") @Test - @Timeout(value = 5, unit = TimeUnit.SECONDS) + @Timeout(value = 5L, unit = TimeUnit.SECONDS) void testDoesNotWaitForPendingRecordsIfFlushingDisabled() throws Throwable { final DummyFlinkKafkaProducer producer = new DummyFlinkKafkaProducer<>( 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 9669df535..fa2bf98c6 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 @@ -65,153 +65,153 @@ protected static void prepare() throws Exception { // ------------------------------------------------------------------------ @Test - @Timeout(value = 120, unit = TimeUnit.SECONDS) - public void testFailOnNoBroker() throws Exception { + @Timeout(value = 120L, unit = TimeUnit.SECONDS) + void testFailOnNoBroker() throws Exception { runFailOnNoBrokerTest(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testConcurrentProducerConsumerTopology() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testConcurrentProducerConsumerTopology() throws Exception { runSimpleConcurrentProducerConsumerTopology(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testKeyValueSupport() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testKeyValueSupport() throws Exception { runKeyValueTest(); } // --- canceling / failures --- @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testCancelingEmptyTopic() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testCancelingEmptyTopic() throws Exception { runCancelingOnEmptyInputTest(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testCancelingFullTopic() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testCancelingFullTopic() throws Exception { runCancelingOnFullInputTest(); } // --- source to partition mappings and exactly once --- @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testOneToOneSources() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testOneToOneSources() throws Exception { runOneToOneExactlyOnceTest(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testOneSourceMultiplePartitions() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testOneSourceMultiplePartitions() throws Exception { runOneSourceMultiplePartitionsExactlyOnceTest(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testMultipleSourcesOnePartition() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testMultipleSourcesOnePartition() throws Exception { runMultipleSourcesOnePartitionExactlyOnceTest(); } // --- broker failure --- @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testBrokerFailure() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testBrokerFailure() throws Exception { runBrokerFailureTest(); } // --- special executions --- @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testBigRecordJob() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testBigRecordJob() throws Exception { runBigRecordTestTopology(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testMultipleTopicsWithLegacySerializer() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testMultipleTopicsWithLegacySerializer() throws Exception { runProduceConsumeMultipleTopics(true); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testMultipleTopicsWithKafkaSerializer() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testMultipleTopicsWithKafkaSerializer() throws Exception { runProduceConsumeMultipleTopics(false); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testAllDeletes() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testAllDeletes() throws Exception { runAllDeletesTest(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testMetricsAndEndOfStream() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testMetricsAndEndOfStream() throws Exception { runEndOfStreamTest(); } // --- startup mode --- @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testStartFromEarliestOffsets() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testStartFromEarliestOffsets() throws Exception { runStartFromEarliestOffsets(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testStartFromLatestOffsets() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testStartFromLatestOffsets() throws Exception { runStartFromLatestOffsets(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testStartFromGroupOffsets() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testStartFromGroupOffsets() throws Exception { runStartFromGroupOffsets(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testStartFromSpecificOffsets() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testStartFromSpecificOffsets() throws Exception { runStartFromSpecificOffsets(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testStartFromTimestamp() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testStartFromTimestamp() throws Exception { runStartFromTimestamp(); } // --- offset committing --- @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testCommitOffsetsToKafka() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testCommitOffsetsToKafka() throws Exception { runCommitOffsetsToKafka(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testAutoOffsetRetrievalAndCommitToKafka() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testAutoOffsetRetrievalAndCommitToKafka() throws Exception { runAutoOffsetRetrievalAndCommitToKafka(); } @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testCollectingSchema() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testCollectingSchema() throws Exception { runCollectingSchemaTest(); } /** Kafka 20 specific test, ensuring Timestamps are properly written to and read from Kafka. */ @Test - @Timeout(value = 60, unit = TimeUnit.SECONDS) - public void testTimestamps() throws Exception { + @Timeout(value = 60L, unit = TimeUnit.SECONDS) + void testTimestamps() throws Exception { final String topic = "tstopic-" + UUID.randomUUID(); createTestTopic(topic, 3, 1); @@ -229,7 +229,7 @@ public void testTimestamps() throws Exception { boolean running = true; @Override - public void run(SourceContext ctx) throws Exception { + public void run(SourceContext ctx) { long i = 0; while (running) { ctx.collectWithTimestamp(i, i * 2); @@ -393,8 +393,7 @@ public Long deserialize(ConsumerRecord record) throws IOExceptio cnt++; DataInputView in = new DataInputViewStreamWrapper(new ByteArrayInputStream(record.value())); - Long e = ser.deserialize(in); - return e; + return ser.deserialize(in); } @Override 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 02bd442f7..8e0c853f1 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 @@ -31,6 +31,7 @@ import org.apache.flink.util.InstantiationUtil; import com.google.common.base.MoreObjects; +import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -38,6 +39,7 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -74,6 +76,7 @@ */ @SuppressWarnings("serial") @RetryOnFailure(times = 3) +@ExtendWith(TestLoggerExtension.class) public abstract class KafkaTestBase { public static final Logger LOG = LoggerFactory.getLogger(KafkaTestBase.class); 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 41230bf77..8d070d0d3 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 @@ -67,10 +67,11 @@ import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; +import org.apache.flink.util.TestLoggerExtension; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; import org.apache.kafka.common.TopicPartition; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; import java.util.Arrays; import java.util.Collections; @@ -80,7 +81,7 @@ import java.util.Properties; import java.util.function.Consumer; -import static org.apache.flink.core.testutils.FlinkMatchers.containsCause; +import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptions.ScanBoundedMode; import static org.apache.flink.streaming.connectors.kafka.table.KafkaConnectorOptionsUtil.AVRO_CONFLUENT; import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink; @@ -89,6 +90,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link UpsertKafkaDynamicTableFactory}. */ +@ExtendWith(TestLoggerExtension.class) class UpsertKafkaDynamicTableFactoryTest { private static final String SOURCE_TOPIC = "sourceTopic_1"; @@ -433,9 +435,7 @@ void testBoundedSpecificOffsetsValidate() { void testBoundedSpecificOffsets() { testBoundedOffsets( ScanBoundedMode.SPECIFIC_OFFSETS, - options -> { - options.put("scan.bounded.specific-offsets", "partition:0,offset:2"); - }, + options -> options.put("scan.bounded.specific-offsets", "partition:0,offset:2"), source -> { assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); OffsetsInitializer offsetsInitializer = @@ -479,9 +479,7 @@ void testBoundedLatestOffset() { void testBoundedGroupOffsets() { testBoundedOffsets( ScanBoundedMode.GROUP_OFFSETS, - options -> { - options.put("properties.group.id", "dummy"); - }, + options -> options.put("properties.group.id", "dummy"), source -> { assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); OffsetsInitializer offsetsInitializer = @@ -501,9 +499,7 @@ void testBoundedGroupOffsets() { void testBoundedTimestamp() { testBoundedOffsets( ScanBoundedMode.TIMESTAMP, - options -> { - options.put("scan.bounded.timestamp-millis", "1"); - }, + options -> options.put("scan.bounded.timestamp-millis", "1"), source -> { assertThat(source.getBoundedness()).isEqualTo(Boundedness.BOUNDED); OffsetsInitializer offsetsInitializer = @@ -549,110 +545,100 @@ void testBoundedTimestamp() { @Test 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. " - + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys."))); - ResolvedSchema illegalSchema = ResolvedSchema.of( Column.physical("window_start", DataTypes.STRING()), Column.physical("region", DataTypes.STRING()), Column.physical("view_count", DataTypes.BIGINT())); - createTableSource(illegalSchema, getFullSourceOptions()); + assertThatThrownBy(() -> createTableSource(illegalSchema, getFullSourceOptions())) + .satisfies(anyCauseMatches(ValidationException.class, + "'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. " + + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys." + )); } @Test 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. " - + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys."))); - ResolvedSchema illegalSchema = ResolvedSchema.of( Column.physical("region", DataTypes.STRING()), Column.physical("view_count", DataTypes.BIGINT())); - createTableSink(illegalSchema, getFullSinkOptions()); + assertThatThrownBy(() -> createTableSink(illegalSchema, getFullSinkOptions())) + .satisfies(anyCauseMatches(ValidationException.class, + "'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. " + + "The PRIMARY KEY also defines records in the 'upsert-kafka' table should update or delete on which keys." + )); } @Test void testSerWithCDCFormatAsValue() { - Assertions.assertThrows(ValidationException.class, - () -> containsCause( - new ValidationException( + assertThatThrownBy(() -> + createTableSink( + SINK_SCHEMA, + getModifiedOptions( + getFullSinkOptions(), + options -> + options.put( + String.format( + "value.%s.%s", + TestFormatFactory.IDENTIFIER, + TestFormatFactory.CHANGELOG_MODE.key()), + "I;UA;UB;D")))) + .satisfies(anyCauseMatches(ValidationException.class, String.format( "'upsert-kafka' connector doesn't support '%s' as value format, " + "because '%s' is not in insert-only mode.", TestFormatFactory.IDENTIFIER, - TestFormatFactory.IDENTIFIER)))); + TestFormatFactory.IDENTIFIER)) + ); - createTableSink( - SINK_SCHEMA, - getModifiedOptions( - getFullSinkOptions(), - options -> - options.put( - String.format( - "value.%s.%s", - TestFormatFactory.IDENTIFIER, - TestFormatFactory.CHANGELOG_MODE.key()), - "I;UA;UB;D"))); } @Test void testDeserWithCDCFormatAsValue() { - Assertions.assertThrows(ValidationException.class, - () -> containsCause( - new ValidationException( + assertThatThrownBy(() -> + createTableSource( + SOURCE_SCHEMA, + getModifiedOptions( + getFullSourceOptions(), + options -> + options.put( + String.format( + "value.%s.%s", + TestFormatFactory.IDENTIFIER, + TestFormatFactory.CHANGELOG_MODE.key()), + "I;UA;UB;D")))) + .satisfies(anyCauseMatches(ValidationException.class, String.format( "'upsert-kafka' connector doesn't support '%s' as value format, " + "because '%s' is not in insert-only mode.", TestFormatFactory.IDENTIFIER, - TestFormatFactory.IDENTIFIER)))); - - createTableSource( - SOURCE_SCHEMA, - getModifiedOptions( - getFullSourceOptions(), - options -> - options.put( - String.format( - "value.%s.%s", - TestFormatFactory.IDENTIFIER, - TestFormatFactory.CHANGELOG_MODE.key()), - "I;UA;UB;D"))); + TestFormatFactory.IDENTIFIER)) + ); } @Test void testInvalidSinkBufferFlush() { - Assertions.assertThrows(ValidationException.class, - () -> containsCause( - new ValidationException( + assertThatThrownBy(() -> + createTableSink( + SINK_SCHEMA, + getModifiedOptions( + getFullSinkOptions(), + options -> { + options.put("sink.buffer-flush.max-rows", "0"); + options.put("sink.buffer-flush.interval", "1s"); + }))) + .satisfies(anyCauseMatches(ValidationException.class, "'sink.buffer-flush.max-rows' and 'sink.buffer-flush.interval' " + "must be set to be greater than zero together to enable" - + " sink buffer flushing."))); - createTableSink( - SINK_SCHEMA, - getModifiedOptions( - getFullSinkOptions(), - options -> { - options.put("sink.buffer-flush.max-rows", "0"); - options.put("sink.buffer-flush.interval", "1s"); - })); + + " sink buffer flushing." + )); } @Test void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { - Assertions.assertThrows(ValidationException.class, - () -> containsCause( - new ValidationException( - "sink.transactional-id-prefix must be specified when using DeliveryGuarantee.EXACTLY_ONCE."))); - final Map modifiedOptions = getModifiedOptions( getFullSinkOptions(), @@ -662,7 +648,10 @@ void testExactlyOnceGuaranteeWithoutTransactionalIdPrefix() { KafkaConnectorOptions.DELIVERY_GUARANTEE.key(), DeliveryGuarantee.EXACTLY_ONCE.toString()); }); - createTableSink(SINK_SCHEMA, modifiedOptions); + assertThatThrownBy(() -> createTableSink(SINK_SCHEMA, modifiedOptions)) + .satisfies(anyCauseMatches(ValidationException.class, + "sink.transactional-id-prefix must be specified when using DeliveryGuarantee.EXACTLY_ONCE." + )); } // -------------------------------------------------------------------------------------------- From 9605a2cb530850058ec86a516b81c658c2d3f587 Mon Sep 17 00:00:00 2001 From: morazow Date: Thu, 20 Jun 2024 01:03:09 +0200 Subject: [PATCH 14/15] fail to assertThatThrownBy migration --- .../kafka/FlinkKafkaConsumerBaseTest.java | 14 +--- .../internals/ClosableBlockingQueueTest.java | 83 +++++-------------- 2 files changed, 22 insertions(+), 75 deletions(-) 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 60cc00495..28682b405 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 @@ -33,6 +33,7 @@ import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.testutils.CheckedThread; +import org.apache.flink.core.testutils.FlinkAssertions; import org.apache.flink.core.testutils.OneShotLatch; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; @@ -60,7 +61,6 @@ import org.apache.flink.streaming.util.MockDeserializationSchema; import org.apache.flink.streaming.util.MockStreamingRuntimeContext; import org.apache.flink.streaming.util.serialization.KeyedDeserializationSchema; -import org.apache.flink.util.ExceptionUtils; import org.apache.flink.util.FlinkException; import org.apache.flink.util.InstantiationUtil; import org.apache.flink.util.Preconditions; @@ -650,18 +650,10 @@ void testClosePartitionDiscovererWhenKafkaFetcherFails() throws Exception { private void testFailingConsumerLifecycle( FlinkKafkaConsumerBase testKafkaConsumer, @Nonnull Exception expectedException) throws Exception { - try { + assertThatThrownBy(() -> { setupConsumer(testKafkaConsumer); testKafkaConsumer.run(new TestSourceContext<>()); - - fail( - "Exception should have been thrown from open / run method of FlinkKafkaConsumerBase."); - } catch (Exception e) { - assertThat( - ExceptionUtils.findThrowable( - e, throwable -> throwable.equals(expectedException))) - .isPresent(); - } + }).satisfies(FlinkAssertions.anyCauseMatches(expectedException.getClass())); testKafkaConsumer.close(); } 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 cd9e9d38d..27f91d510 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 @@ -27,6 +27,7 @@ import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.fail; /** Tests for the {@link ClosableBlockingQueue}. */ @@ -46,10 +47,10 @@ void testCreateQueueHashCodeEquals() { assertThat(queue2.isOpen()).isTrue(); assertThat(queue1.isEmpty()).isTrue(); assertThat(queue2.isEmpty()).isTrue(); - assertThat(queue1.size()).isEqualTo(0); - assertThat(queue2.size()).isEqualTo(0); + assertThat(queue1.size()).isZero(); + assertThat(queue2.size()).isZero(); - assertThat(queue1.hashCode()).isEqualTo(queue2.hashCode()); + assertThat(queue1).hasSameHashCodeAs(queue2); //noinspection EqualsWithItself assertThat(queue1.equals(queue1)).isTrue(); //noinspection EqualsWithItself @@ -101,12 +102,7 @@ void testCloseEmptyQueue() { assertThat(queue.addIfOpen("element")).isFalse(); assertThat(queue.isEmpty()).isTrue(); - try { - queue.add("some element"); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } + assertThatThrownBy(() -> queue.add("some element")).isInstanceOf(IllegalStateException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -135,12 +131,7 @@ void testCloseNonEmptyQueue() { assertThat(queue.addIfOpen(42)).isFalse(); assertThat(queue.isEmpty()).isTrue(); - try { - queue.add(99); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } + assertThatThrownBy(() -> queue.add(99)).isInstanceOf(IllegalStateException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -157,7 +148,7 @@ void testPeekAndPoll() { assertThat(queue.poll()).isNull(); assertThat(queue.poll()).isNull(); - assertThat(queue.size()).isEqualTo(0); + assertThat(queue.size()).isZero(); queue.add("a"); queue.add("b"); @@ -181,26 +172,15 @@ void testPeekAndPoll() { assertThat(queue.poll()).isEqualTo("c"); - assertThat(queue.size()).isEqualTo(0); + assertThat(queue.size()).isZero(); assertThat(queue.poll()).isNull(); assertThat(queue.peek()).isNull(); assertThat(queue.peek()).isNull(); assertThat(queue.close()).isTrue(); - try { - queue.peek(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } - - try { - queue.poll(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } + assertThatThrownBy(() -> queue.peek()).isInstanceOf(IllegalStateException.class); + assertThatThrownBy(() -> queue.poll()).isInstanceOf(IllegalStateException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -227,12 +207,7 @@ void testPollBatch() { assertThat(queue.close()).isTrue(); - try { - queue.pollBatch(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } + assertThatThrownBy(() -> queue.pollBatch()).isInstanceOf(IllegalStateException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -248,7 +223,7 @@ void testGetElementBlocking() { assertThat(queue.getElementBlocking(3)).isNull(); assertThat(queue.getElementBlocking(2)).isNull(); - assertThat(queue.size()).isEqualTo(0); + assertThat(queue.size()).isZero(); queue.add("a"); queue.add("b"); @@ -269,7 +244,7 @@ void testGetElementBlocking() { assertThat(queue.getElementBlocking()).isEqualTo("e"); assertThat(queue.getElementBlocking(1786598)).isEqualTo("f"); - assertThat(queue.size()).isEqualTo(0); + assertThat(queue.size()).isZero(); assertThat(queue.getElementBlocking(1)).isNull(); assertThat(queue.getElementBlocking(3)).isNull(); @@ -277,19 +252,9 @@ void testGetElementBlocking() { assertThat(queue.close()).isTrue(); - try { - queue.getElementBlocking(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } - - try { - queue.getElementBlocking(1000000000L); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } + assertThatThrownBy(() -> queue.getElementBlocking()).isInstanceOf(IllegalStateException.class); + assertThatThrownBy(() -> queue.getElementBlocking(1000000000L)) + .isInstanceOf(IllegalStateException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); @@ -333,19 +298,9 @@ void testGetBatchBlocking() { assertThat(queue.close()).isTrue(); - try { - queue.getBatchBlocking(); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } - - try { - queue.getBatchBlocking(1000000000L); - fail("should cause an exception"); - } catch (IllegalStateException ignored) { - // expected - } + assertThatThrownBy(() -> queue.getBatchBlocking()).isInstanceOf(IllegalStateException.class); + assertThatThrownBy(() -> queue.getBatchBlocking(1000000000L)) + .isInstanceOf(IllegalStateException.class); } catch (Exception e) { e.printStackTrace(); fail(e.getMessage()); From 4128b55317ca3c84964544dbd1200bcda6e9312a Mon Sep 17 00:00:00 2001 From: morazow Date: Thu, 20 Jun 2024 01:35:26 +0200 Subject: [PATCH 15/15] Rebase main --- .../kafka/sink/KafkaWriterITCase.java | 159 +++++++++--------- .../connectors/kafka/KafkaTestBase.java | 36 ++-- 2 files changed, 96 insertions(+), 99 deletions(-) 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 f2043cfdb..616d85a92 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 @@ -27,9 +27,11 @@ import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.OperatorMetricGroup; import org.apache.flink.metrics.groups.SinkWriterMetricGroup; import org.apache.flink.metrics.testutils.MetricListener; import org.apache.flink.runtime.metrics.groups.InternalSinkWriterMetricGroup; +import org.apache.flink.runtime.metrics.groups.ProxyMetricGroup; import org.apache.flink.runtime.metrics.groups.UnregisteredMetricGroups; import org.apache.flink.util.TestLoggerExtension; import org.apache.flink.util.UserCodeClassLoader; @@ -58,7 +60,6 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Optional; @@ -69,9 +70,9 @@ import java.util.function.Consumer; import java.util.stream.IntStream; +import static org.apache.flink.connector.kafka.testutils.DockerImageVersions.KAFKA; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.createKafkaContainer; import static org.apache.flink.connector.kafka.testutils.KafkaUtil.drainAllRecordsFromTopic; -import static org.apache.flink.util.DockerImageVersions.KAFKA; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.AssertionsForClassTypes.assertThatCode; @@ -84,7 +85,7 @@ class KafkaWriterITCase { private static final Network NETWORK = Network.newNetwork(); private static final String KAFKA_METRIC_WITH_GROUP_NAME = "KafkaProducer.incoming-byte-total"; private static final SinkWriter.Context SINK_WRITER_CONTEXT = new DummySinkWriterContext(); - private String topic; + private static String topic; private MetricListener metricListener; private TriggerTimeService timeService; @@ -114,7 +115,7 @@ void setUp(TestInfo testInfo) { @ParameterizedTest @EnumSource(DeliveryGuarantee.class) - void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { + public void testRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { try (final KafkaWriter ignored = createWriterWithConfiguration(getKafkaClientConfiguration(), guarantee)) { assertThat(metricListener.getGauge(KAFKA_METRIC_WITH_GROUP_NAME)).isPresent(); @@ -130,11 +131,8 @@ void testNotRegisterMetrics(DeliveryGuarantee guarantee) throws Exception { @Test void testIncreasingRecordBasedCounters() throws Exception { - final OperatorIOMetricGroup operatorIOMetricGroup = - UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup().getIOMetricGroup(); - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock( - metricListener.getMetricGroup(), operatorIOMetricGroup); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + try (final KafkaWriter writer = createWriterWithConfiguration( getKafkaClientConfiguration(), DeliveryGuarantee.NONE, metricGroup)) { @@ -166,14 +164,10 @@ void testIncreasingRecordBasedCounters() throws Exception { } @Test - void testCurrentSendTimeMetric() throws Exception { - final InternalSinkWriterMetricGroup metricGroup = - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()); + public void testCurrentSendTimeMetric() throws Exception { try (final KafkaWriter writer = createWriterWithConfiguration( - getKafkaClientConfiguration(), - DeliveryGuarantee.AT_LEAST_ONCE, - metricGroup)) { + getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE)) { final Optional> currentSendTime = metricListener.getGauge("currentSendTime"); assertThat(currentSendTime).isPresent(); @@ -199,16 +193,12 @@ void testCurrentSendTimeMetric() throws Exception { void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isZero(); triggerProducerException(writer, properties); @@ -228,16 +218,12 @@ void testFlushAsyncErrorPropagationAndErrorCounter() throws Exception { void testWriteAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isZero(); triggerProducerException(writer, properties); @@ -259,10 +245,8 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + new SinkInitContext(createSinkWriterMetricGroup(), timeService, null); + final KafkaWriter writer = createWriterWithConfiguration( properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); @@ -289,16 +273,12 @@ void testMailboxAsyncErrorPropagationAndErrorCounter() throws Exception { void testCloseAsyncErrorPropagationAndErrorCounter() throws Exception { Properties properties = getKafkaClientConfiguration(); - SinkInitContext sinkInitContext = - new SinkInitContext( - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), - timeService, - null); + final SinkWriterMetricGroup metricGroup = createSinkWriterMetricGroup(); + final KafkaWriter writer = createWriterWithConfiguration( - properties, DeliveryGuarantee.EXACTLY_ONCE, sinkInitContext); - final Counter numRecordsOutErrors = - sinkInitContext.metricGroup.getNumRecordsOutErrorsCounter(); + properties, DeliveryGuarantee.EXACTLY_ONCE, metricGroup); + final Counter numRecordsOutErrors = metricGroup.getNumRecordsOutErrorsCounter(); assertThat(numRecordsOutErrors.getCount()).isZero(); triggerProducerException(writer, properties); @@ -334,7 +314,7 @@ void testMetadataPublisher() throws Exception { createWriterWithConfiguration( getKafkaClientConfiguration(), DeliveryGuarantee.AT_LEAST_ONCE, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup()), + createSinkWriterMetricGroup(), meta -> metadataList.add(meta.toString()))) { List expected = new ArrayList<>(); for (int i = 0; i < 100; i++) { @@ -413,7 +393,7 @@ void usePoolForTransactional() throws Exception { try (final KafkaWriter writer = createWriterWithConfiguration( getKafkaClientConfiguration(), DeliveryGuarantee.EXACTLY_ONCE)) { - assertThat(writer.getProducerPool()).isEmpty(); + assertThat(writer.getProducerPool()).hasSize(0); writer.write(1, SINK_WRITER_CONTEXT); writer.flush(false); @@ -425,7 +405,7 @@ void usePoolForTransactional() throws Exception { FlinkKafkaInternalProducer firstProducer = committable.getProducer().get().getObject(); - assertThat(firstProducer) + assertThat(firstProducer != writer.getCurrentProducer()) .as("Expected different producer") .isNotSameAs(writer.getCurrentProducer()); @@ -518,17 +498,15 @@ private void assertKafkaMetricNotPresent( } private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee) { - return createWriterWithConfiguration( - config, - guarantee, - InternalSinkWriterMetricGroup.mock(metricListener.getMetricGroup())); + Properties config, DeliveryGuarantee guarantee) throws IOException { + return createWriterWithConfiguration(config, guarantee, createSinkWriterMetricGroup()); } private KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee, - SinkWriterMetricGroup sinkWriterMetricGroup) { + SinkWriterMetricGroup sinkWriterMetricGroup) + throws IOException { return createWriterWithConfiguration(config, guarantee, sinkWriterMetricGroup, null); } @@ -536,27 +514,37 @@ private KafkaWriter createWriterWithConfiguration( Properties config, DeliveryGuarantee guarantee, SinkWriterMetricGroup sinkWriterMetricGroup, - @Nullable Consumer metadataConsumer) { - return new KafkaWriter<>( - guarantee, - config, - "test-prefix", - new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer), - new DummyRecordSerializer(), - new DummySchemaContext(), - Collections.emptyList()); + @Nullable Consumer metadataConsumer) + throws IOException { + KafkaSink kafkaSink = + KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); + return (KafkaWriter) + kafkaSink.createWriter( + new SinkInitContext(sinkWriterMetricGroup, timeService, metadataConsumer)); } private KafkaWriter createWriterWithConfiguration( - Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) { - return new KafkaWriter<>( - guarantee, - config, - "test-prefix", - sinkInitContext, - new DummyRecordSerializer(), - new DummySchemaContext(), - Collections.emptyList()); + Properties config, DeliveryGuarantee guarantee, SinkInitContext sinkInitContext) + throws IOException { + KafkaSink kafkaSink = + KafkaSink.builder() + .setKafkaProducerConfig(config) + .setDeliveryGuarantee(guarantee) + .setTransactionalIdPrefix("test-prefix") + .setRecordSerializer(new DummyRecordSerializer()) + .build(); + return (KafkaWriter) kafkaSink.createWriter(sinkInitContext); + } + + private SinkWriterMetricGroup createSinkWriterMetricGroup() { + DummyOperatorMetricGroup operatorMetricGroup = + new DummyOperatorMetricGroup(metricListener.getMetricGroup()); + return InternalSinkWriterMetricGroup.wrap(operatorMetricGroup); } private static Properties getKafkaClientConfiguration() { @@ -632,7 +620,7 @@ public Optional> metadataConsumer() { } } - private class DummyRecordSerializer implements KafkaRecordSerializationSchema { + private static class DummyRecordSerializer implements KafkaRecordSerializationSchema { @Override public ProducerRecord serialize( Integer element, KafkaSinkContext context, Long timestamp) { @@ -644,28 +632,33 @@ public ProducerRecord serialize( } } - private static class DummySchemaContext implements SerializationSchema.InitializationContext { - + private static class DummySinkWriterContext implements SinkWriter.Context { @Override - public MetricGroup getMetricGroup() { - throw new UnsupportedOperationException("Not implemented."); + public long currentWatermark() { + return 0; } @Override - public UserCodeClassLoader getUserCodeClassLoader() { - throw new UnsupportedOperationException("Not implemented."); + public Long timestamp() { + return null; } } - private static class DummySinkWriterContext implements SinkWriter.Context { - @Override - public long currentWatermark() { - return 0; + private static class DummyOperatorMetricGroup extends ProxyMetricGroup + implements OperatorMetricGroup { + + private final OperatorIOMetricGroup operatorIOMetricGroup; + + public DummyOperatorMetricGroup(MetricGroup parentMetricGroup) { + super(parentMetricGroup); + this.operatorIOMetricGroup = + UnregisteredMetricGroups.createUnregisteredOperatorMetricGroup() + .getIOMetricGroup(); } @Override - public Long timestamp() { - return null; + public OperatorIOMetricGroup getIOMetricGroup() { + return operatorIOMetricGroup; } } 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 8e0c853f1..6dacb1e17 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 @@ -147,8 +147,9 @@ public static void startClusters( KafkaTestEnvironment.Config environmentConfig, int numKafkaClusters) throws Exception { for (int i = 0; i < numKafkaClusters; i++) { startClusters(environmentConfig); - KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = new KafkaClusterTestEnvMetadata( - i, kafkaServer, standardProps, brokerConnectionStrings, secureProps); + KafkaClusterTestEnvMetadata kafkaClusterTestEnvMetadata = + new KafkaClusterTestEnvMetadata( + i, kafkaServer, standardProps, brokerConnectionStrings, secureProps); kafkaClusters.add(kafkaClusterTestEnvMetadata); LOG.info("Created Kafka cluster with configuration: {}", kafkaClusterTestEnvMetadata); } @@ -176,8 +177,9 @@ public static void startClusters(KafkaTestEnvironment.Config environmentConfig) } public static KafkaTestEnvironment constructKafkaTestEnvironment() throws Exception { - Class clazz = Class.forName( - "org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); + Class clazz = + Class.forName( + "org.apache.flink.streaming.connectors.kafka.KafkaTestEnvironmentImpl"); return (KafkaTestEnvironment) InstantiationUtil.instantiate(clazz); } @@ -233,7 +235,8 @@ public static void deleteTestTopic(String topic) { public static void produceToKafka( Collection> records, Class> keySerializerClass, - Class> valueSerializerClass) + Class> + valueSerializerClass) throws Throwable { Properties props = new Properties(); props.putAll(standardProps); @@ -243,13 +246,14 @@ public static void produceToKafka( ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, valueSerializerClass.getName()); AtomicReference sendingError = new AtomicReference<>(); - Callback callback = (metadata, exception) -> { - if (exception != null) { - if (!sendingError.compareAndSet(null, exception)) { - sendingError.get().addSuppressed(exception); - } - } - }; + Callback callback = + (metadata, exception) -> { + if (exception != null) { + if (!sendingError.compareAndSet(null, exception)) { + sendingError.get().addSuppressed(exception); + } + } + }; try (KafkaProducer producer = new KafkaProducer<>(props)) { for (ProducerRecord record : records) { producer.send(record, callback); @@ -291,8 +295,8 @@ public void assertAtLeastOnceForTopic( properties.put("heartbeat.interval.ms", "500"); // query kafka for new records ... - Collection> records = kafkaServer.getAllRecordsFromTopic(properties, - topic); + Collection> records = + kafkaServer.getAllRecordsFromTopic(properties, topic); for (ConsumerRecord record : records) { actualElements.add(record.value()); @@ -324,8 +328,8 @@ public void assertExactlyOnceForTopic( consumerProperties.put("isolation.level", "read_committed"); // query kafka for new records ... - Collection> records = kafkaServer.getAllRecordsFromTopic(consumerProperties, - topic); + Collection> records = + kafkaServer.getAllRecordsFromTopic(consumerProperties, topic); for (ConsumerRecord record : records) { actualElements.add(ByteBuffer.wrap(record.value()).getInt());