AHeise commented on code in PR #195:
URL: 
https://github.com/apache/flink-connector-kafka/pull/195#discussion_r2527541576


##########
flink-connector-kafka/src/test/java/org/apache/flink/connector/kafka/source/KafkaSourceMigrationITCase.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.connector.kafka.source;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.eventtime.WatermarkStrategy;
+import org.apache.flink.api.common.typeinfo.TypeHint;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.client.program.ClusterClient;
+import org.apache.flink.configuration.Configuration;
+import 
org.apache.flink.connector.kafka.source.reader.deserializer.KafkaRecordDeserializationSchema;
+import org.apache.flink.connector.kafka.testutils.KafkaSourceTestEnv;
+import org.apache.flink.core.execution.JobClient;
+import org.apache.flink.core.execution.SavepointFormatType;
+import org.apache.flink.runtime.clusterframework.ApplicationStatus;
+import org.apache.flink.runtime.jobmaster.JobResult;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.test.junit5.InjectClusterClient;
+import org.apache.flink.test.junit5.InjectMiniCluster;
+import org.apache.flink.test.junit5.MiniClusterExtension;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.apache.kafka.clients.consumer.ConsumerRecord;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.assertj.core.api.InstanceOfAssertFactories;
+import org.assertj.core.api.SoftAssertions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+import org.testcontainers.junit.jupiter.Testcontainers;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Stream;
+
+import static 
org.apache.flink.configuration.StateRecoveryOptions.SAVEPOINT_IGNORE_UNCLAIMED_STATE;
+import static 
org.apache.flink.configuration.StateRecoveryOptions.SAVEPOINT_PATH;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** The test for creation savepoint for migration tests for the Kafka Sink. */
+@Testcontainers
+public class KafkaSourceMigrationITCase extends TestLogger {
+    public static final String KAFKA_SOURCE_UID = "kafka-source-operator-uid";
+    // Directory to store the savepoints in src/test/resources
+    private static final Path KAFKA_SOURCE_SAVEPOINT_PATH =
+            
Path.of("src/test/resources/kafka-source-savepoint").toAbsolutePath();
+
+    @RegisterExtension
+    public static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
+            new MiniClusterExtension(
+                    new MiniClusterResourceConfiguration.Builder()
+                            .setNumberTaskManagers(2)
+                            .setNumberSlotsPerTaskManager(3)
+                            .build());
+
+    public static final int NUM_RECORDS =
+            KafkaSourceTestEnv.NUM_PARTITIONS * 
KafkaSourceTestEnv.NUM_RECORDS_PER_PARTITION;
+    private static final String TOPIC = "topic";
+
+    @BeforeEach
+    void setupEnv() throws Throwable {
+        KafkaSourceTestEnv.setup();
+    }
+
+    @AfterEach
+    void removeEnv() throws Exception {
+        KafkaSourceTestEnv.tearDown();
+    }
+
+    static Stream<Arguments> getKafkaSourceSavepoint() throws IOException {
+        return Files.walk(KAFKA_SOURCE_SAVEPOINT_PATH)
+                .filter(
+                        f ->
+                                Files.isDirectory(f)
+                                        && 
f.getFileName().toString().startsWith("savepoint"))
+                // allow
+                .map(KAFKA_SOURCE_SAVEPOINT_PATH::relativize)
+                .map(Arguments::arguments);
+    }
+
+    @Disabled("Enable if you want to create savepoint of KafkaSource")
+    @Test
+    void createAndStoreSavepoint(
+            @InjectMiniCluster MiniCluster miniCluster,
+            @InjectClusterClient ClusterClient<?> clusterClient)
+            throws Throwable {
+
+        // this is the part that has been read already in the savepoint
+        KafkaSourceTestEnv.createTestTopic(TOPIC);
+        final List<ProducerRecord<String, Integer>> writtenRecords =
+                KafkaSourceTestEnv.getRecordsForTopic(TOPIC);
+        KafkaSourceTestEnv.produceToKafka(writtenRecords);
+
+        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setParallelism(3);
+
+        final KafkaSource<TimestampedRecord> source = createSource();
+        final int enumVersion = 
source.getEnumeratorCheckpointSerializer().getVersion();
+        final int splitVersion = source.getSplitSerializer().getVersion();
+        String testCase = String.format("enum%s-split%s", enumVersion, 
splitVersion);
+
+        Path savepointPath = KAFKA_SOURCE_SAVEPOINT_PATH.resolve(testCase);
+        Files.createDirectories(savepointPath);
+
+        final CloseableIterator<TimestampedRecord> recordIterator =
+                env.fromSource(source, WatermarkStrategy.noWatermarks(), 
"TestDataSource")
+                        .uid(KAFKA_SOURCE_UID)
+                        .collectAsync();
+
+        final JobClient jobClient = env.executeAsync();
+        final JobID jobID = jobClient.getJobID();
+
+        final Queue<TimestampedRecord> readRecords = new 
ConcurrentLinkedQueue<>();
+        while (readRecords.size() < writtenRecords.size() && 
recordIterator.hasNext()) {
+            readRecords.add(recordIterator.next());
+        }
+
+        try {
+            CompletableFuture<String> savepointFuture =
+                    clusterClient.stopWithSavepoint(
+                            jobID, false, savepointPath.toString(), 
SavepointFormatType.NATIVE);
+            savepointFuture.get(2, TimeUnit.MINUTES);
+
+            final long maxTS = getMaxTS(writtenRecords);
+            assertThat(readRecords).hasSize(NUM_RECORDS).allMatch(r -> 
r.getTimestamp() <= maxTS);
+        } finally {
+            SoftAssertions softly = new SoftAssertions();
+            softly.assertThat(clusterClient.requestJobResult(jobID).get())
+                    .returns(ApplicationStatus.SUCCEEDED, 
JobResult::getApplicationStatus)
+                    .extracting(
+                            JobResult::getSerializedThrowable,
+                            
InstanceOfAssertFactories.optional(Exception.class))
+                    .isEmpty();
+            softly.assertAll();
+        }
+    }
+
+    private static long getMaxTS(List<ProducerRecord<String, Integer>> 
writtenRecords) {
+        return 
writtenRecords.stream().mapToLong(ProducerRecord::timestamp).max().orElseThrow();
+    }
+
+    private static KafkaSource<TimestampedRecord> createSource() {
+        return KafkaSource.<TimestampedRecord>builder()
+                
.setBootstrapServers(KafkaSourceTestEnv.brokerConnectionStrings)
+                .setTopics(TOPIC)
+                .setDeserializer(new TestDeserializer())
+                .build();
+    }
+
+    @ParameterizedTest(name = "{0}")
+    @MethodSource("getKafkaSourceSavepoint")
+    void testRestoreFromSavepointWithCurrentVersion(Path savepointPath) throws 
Throwable {
+        // this is the part that has been read already in the savepoint
+        KafkaSourceTestEnv.createTestTopic(TOPIC);
+        final List<ProducerRecord<String, Integer>> existingRecords =
+                KafkaSourceTestEnv.getRecordsForTopic(TOPIC);
+        KafkaSourceTestEnv.produceToKafka(existingRecords);
+        // the new data supposed to be read after resuming from the savepoint
+        final List<ProducerRecord<String, Integer>> writtenRecords =
+                KafkaSourceTestEnv.getRecordsForTopicWithoutTimestamp(TOPIC);
+        KafkaSourceTestEnv.produceToKafka(writtenRecords);
+
+        final Configuration configuration = new Configuration();
+        configuration.set(
+                SAVEPOINT_PATH, 
KAFKA_SOURCE_SAVEPOINT_PATH.resolve(savepointPath).toString());
+        configuration.set(SAVEPOINT_IGNORE_UNCLAIMED_STATE, true);

Review Comment:
   That was needed because the collectAsync also retains state internally which 
we don't recover. I now just read into a concurrent collection and thus it's 
not needed anymore.



-- 
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