fapaul commented on a change in pull request #16676:
URL: https://github.com/apache/flink/pull/16676#discussion_r684155271



##########
File path: 
flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/sink/KafkaSinkITCase.java
##########
@@ -0,0 +1,634 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.streaming.connectors.kafka.sink;
+
+import org.apache.flink.api.common.functions.MapFunction;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.api.common.state.ListState;
+import org.apache.flink.api.common.state.ListStateDescriptor;
+import org.apache.flink.api.common.typeutils.SimpleTypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot;
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton;
+import org.apache.flink.configuration.CheckpointingOptions;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.CoreOptions;
+import org.apache.flink.configuration.StateBackendOptions;
+import org.apache.flink.connector.base.DeliveryGuarantee;
+import org.apache.flink.core.memory.DataInputView;
+import org.apache.flink.core.memory.DataOutputView;
+import org.apache.flink.runtime.jobgraph.SavepointConfigOptions;
+import org.apache.flink.runtime.state.FunctionInitializationContext;
+import org.apache.flink.runtime.state.FunctionSnapshotContext;
+import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.CheckpointConfig;
+import 
org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
+import org.apache.flink.streaming.api.environment.LocalStreamEnvironment;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.SourceFunction;
+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.util.TestLogger;
+
+import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableMap;
+
+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.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.serialization.ByteArrayDeserializer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.testcontainers.containers.KafkaContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.output.Slf4jLogConsumer;
+import org.testcontainers.utility.DockerImageName;
+
+import javax.annotation.Nullable;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.hasItems;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/** Tests for using KafkaSink writing to a Kafka cluster. */
+public class KafkaSinkITCase extends TestLogger {
+
+    private static final Logger LOG = 
LoggerFactory.getLogger(KafkaSinkITCase.class);
+    private static final Slf4jLogConsumer LOG_CONSUMER = new 
Slf4jLogConsumer(LOG);
+    private static final String INTER_CONTAINER_KAFKA_ALIAS = "kafka";
+    private static final Network NETWORK = Network.newNetwork();
+    private static final int ZK_TIMEOUT_MILLIS = 30000;
+    private static final short TOPIC_REPLICATION_FACTOR = 1;
+    private static final Duration CONSUMER_POLL_DURATION = 
Duration.ofSeconds(1);
+
+    private String topic;
+    private SharedReference<AtomicLong> emittedRecordsCount;
+    private SharedReference<AtomicLong> emittedRecordsWithCheckpoint;
+    private SharedReference<AtomicBoolean> failed;
+    private SharedReference<AtomicLong> lastCheckpointedRecord;
+
+    @ClassRule
+    public static final KafkaContainer KAFKA_CONTAINER =
+            new 
KafkaContainer(DockerImageName.parse("confluentinc/cp-kafka:5.5.2"))
+                    .withEmbeddedZookeeper()
+                    .withEnv(
+                            ImmutableMap.of(
+                                    
"KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR",
+                                    "1",
+                                    "KAFKA_TRANSACTION_MAX_TIMEOUT_MS",
+                                    
String.valueOf(Duration.ofHours(2).toMillis()),
+                                    "KAFKA_TRANSACTION_STATE_LOG_MIN_ISR",
+                                    "1",
+                                    "KAFKA_MIN_INSYNC_REPLICAS",
+                                    "1"))
+                    .withNetwork(NETWORK)
+                    .withLogConsumer(LOG_CONSUMER)
+                    .withNetworkAliases(INTER_CONTAINER_KAFKA_ALIAS);
+
+    @Rule public final SharedObjects sharedObjects = SharedObjects.create();
+
+    @Rule public final TemporaryFolder temp = new TemporaryFolder();
+
+    @Before
+    public void setUp() throws ExecutionException, InterruptedException, 
TimeoutException {
+        emittedRecordsCount = sharedObjects.add(new AtomicLong());
+        emittedRecordsWithCheckpoint = sharedObjects.add(new AtomicLong());
+        failed = sharedObjects.add(new AtomicBoolean(false));
+        lastCheckpointedRecord = sharedObjects.add(new AtomicLong(0));
+        topic = UUID.randomUUID().toString();
+        createTestTopic(topic, 1, TOPIC_REPLICATION_FACTOR);
+    }
+
+    @After
+    public void tearDown() throws ExecutionException, InterruptedException, 
TimeoutException {
+        deleteTestTopic(topic);
+    }
+
+    @Test
+    public void testWriteRecordsToKafkaWithAtLeastOnceGuarantee() throws 
Exception {
+        writeRecordsToKafka(DeliveryGuarantee.AT_LEAST_ONCE, 
emittedRecordsCount);
+    }
+
+    @Test
+    public void testWriteRecordsToKafkaWithNoneGuarantee() throws Exception {
+        writeRecordsToKafka(DeliveryGuarantee.NONE, emittedRecordsCount);
+    }
+
+    @Test
+    public void testWriteRecordsToKafkaWithExactlyOnceGuarantee() throws 
Exception {
+        writeRecordsToKafka(DeliveryGuarantee.EXACTLY_ONCE, 
emittedRecordsWithCheckpoint);
+    }
+
+    @Test
+    public void testRecoveryWithAtLeastOnceGuarantee() throws Exception {
+        testRecoveryWithAssertion(
+                DeliveryGuarantee.AT_LEAST_ONCE,
+                (records) ->
+                        assertThat(records, hasItems(1L, 2L, 3L, 4L, 5L, 6L, 
7L, 8L, 9L, 10L)));
+    }
+
+    @Test
+    public void testRecoveryWithExactlyOnceGuarantee() throws Exception {
+        testRecoveryWithAssertion(
+                DeliveryGuarantee.EXACTLY_ONCE,
+                (records) ->
+                        assertEquals(
+                                records,
+                                LongStream.range(1, 
lastCheckpointedRecord.get().get() + 1)
+                                        .boxed()
+                                        .collect(Collectors.toList())));
+    }
+
+    @Test
+    public 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();
+        config.setString(
+                CheckpointingOptions.CHECKPOINTS_DIRECTORY, 
checkpointDir.toURI().toString());
+        config.set(
+                ExecutionCheckpointingOptions.EXTERNALIZED_CHECKPOINT,
+                
CheckpointConfig.ExternalizedCheckpointCleanup.RETAIN_ON_CANCELLATION);
+        config.set(CheckpointingOptions.MAX_RETAINED_CHECKPOINTS, 2);
+        try {
+            executeWithMapper(new FailAsyncCheckpointMapper(1), config, 
"firstPrefix");
+        } catch (Exception e) {
+            assertThat(
+                    e.getCause().getCause().getMessage(),
+                    containsString("Exceeded checkpoint tolerable failure"));
+        }
+        final File completedCheckpoint = 
TestUtils.getMostRecentCompletedCheckpoint(checkpointDir);
+
+        config.set(SavepointConfigOptions.SAVEPOINT_PATH, 
completedCheckpoint.toURI().toString());
+
+        // Run a second job which aborts all lingering transactions and new 
consumer should
+        // immediately see the newly written records
+        failed.get().set(true);
+        executeWithMapper(
+                new FailingCheckpointMapper(failed, lastCheckpointedRecord), 
config, "newPrefix");
+        final List<ConsumerRecord<byte[], byte[]>> collectedRecords =
+                drainAllRecordsFromTopic(topic);
+        assertEquals(
+                deserializeValues(collectedRecords),
+                LongStream.range(1, lastCheckpointedRecord.get().get() + 1)
+                        .boxed()
+                        .collect(Collectors.toList()));
+    }
+
+    @Test
+    public 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);
+        try {
+            executeWithMapper(new FailAsyncCheckpointMapper(0), config, null);
+        } catch (Exception e) {
+            assertThat(
+                    e.getCause().getCause().getMessage(),
+                    containsString("Exceeded checkpoint tolerable failure"));
+        }
+        
assertTrue(deserializeValues(drainAllRecordsFromTopic(topic)).isEmpty());
+
+        // Second job aborts all transactions from previous runs with higher 
parallelism
+        config.set(CoreOptions.DEFAULT_PARALLELISM, 1);
+        failed.get().set(true);
+        executeWithMapper(
+                new FailingCheckpointMapper(failed, lastCheckpointedRecord), 
config, null);
+        final List<ConsumerRecord<byte[], byte[]>> collectedRecords =
+                drainAllRecordsFromTopic(topic);
+        assertEquals(
+                deserializeValues(collectedRecords),
+                LongStream.range(1, lastCheckpointedRecord.get().get() + 1)
+                        .boxed()
+                        .collect(Collectors.toList()));
+    }
+
+    private void executeWithMapper(
+            MapFunction<Long, Long> mapper,
+            Configuration config,
+            @Nullable String transactionalIdPrefix)
+            throws Exception {
+        final StreamExecutionEnvironment env = new 
LocalStreamEnvironment(config);
+        env.enableCheckpointing(100L);
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        final DataStreamSource<Long> source = env.fromSequence(1, 10);
+        final DataStream<Long> stream = source.map(mapper);
+        final KafkaSinkBuilder<Long> builder =
+                new KafkaSinkBuilder<Long>()
+                        .setKafkaProducerConfig(getKafkaClientConfiguration())
+                        .setDeliverGuarantee(DeliveryGuarantee.EXACTLY_ONCE)
+                        
.setBootstrapServers(KAFKA_CONTAINER.getBootstrapServers())
+                        .setRecordSerializer(new RecordSerializer(topic));
+        if (transactionalIdPrefix == null) {
+            transactionalIdPrefix = "kafka-sink";
+        }
+        builder.setTransactionalIdPrefix(transactionalIdPrefix);
+        stream.sinkTo(builder.build());
+        env.execute();
+        checkProducerLeak();
+    }
+
+    private void testRecoveryWithAssertion(
+            DeliveryGuarantee guarantee, 
java.util.function.Consumer<List<Long>> recordsAssertion)
+            throws Exception {

Review comment:
       Thanks for the suggestion, I get the idea. I will try to facilitate it 
next time or do you want to have the tests refactored?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to