This is an automated email from the ASF dual-hosted git repository.

arvid pushed a commit to branch release-1.12
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.12 by this push:
     new 6f4bc8c  [FLINK-20418][core] Fixing checkpointing of 
IteratorSourceReader.
6f4bc8c is described below

commit 6f4bc8c9571f54788b383ef76beb5615e2cdb0e5
Author: Arvid Heise <[email protected]>
AuthorDate: Tue Dec 1 14:37:10 2020 +0100

    [FLINK-20418][core] Fixing checkpointing of IteratorSourceReader.
    
    IteratorSourceReader fails checkpointing when assignment has not yet 
happened with NPE.
    Furthermore, if the last element of a split has been pulled just before 
checkpointing, then a split would be created with (to+1, to) leading to an 
IllegalArgumentException: 'from' must be <= 'to'.
---
 .../source/lib/util/IteratorSourceReader.java      |  7 ++-
 .../source/lib/NumberSequenceSourceITCase.java     | 59 ++++++++++++++++++++++
 2 files changed, 65 insertions(+), 1 deletion(-)

diff --git 
a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java
 
b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java
index 92ab054..d16e597 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/connector/source/lib/util/IteratorSourceReader.java
@@ -27,6 +27,7 @@ import javax.annotation.Nullable;
 
 import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Queue;
@@ -129,8 +130,12 @@ public class IteratorSourceReader<E, IterT extends 
Iterator<E>, SplitT extends I
 
        @Override
        public List<SplitT> snapshotState(long checkpointId) {
+               if (remainingSplits == null) {
+                       // no assignment yet
+                       return Collections.emptyList();
+               }
                final ArrayList<SplitT> allSplits = new ArrayList<>(1 + 
remainingSplits.size());
-               if (iterator != null) {
+               if (iterator != null && iterator.hasNext()) {
                        @SuppressWarnings("unchecked")
                        final SplitT inProgressSplit = (SplitT) 
currentSplit.getUpdatedSplitForIterator(iterator);
                        allSplits.add(inProgressSplit);
diff --git 
a/flink-tests/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceITCase.java
 
b/flink-tests/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceITCase.java
new file mode 100644
index 0000000..764fb6a
--- /dev/null
+++ 
b/flink-tests/src/test/java/org/apache/flink/api/connector/source/lib/NumberSequenceSourceITCase.java
@@ -0,0 +1,59 @@
+/*
+ * 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.api.connector.source.lib;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.util.StreamCollector;
+
+import org.junit.Rule;
+import org.junit.Test;
+
+import java.util.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.LongStream;
+
+import static org.junit.Assert.assertArrayEquals;
+
+/**
+ * Tests {@link NumberSequenceSource}.
+ */
+public class NumberSequenceSourceITCase {
+       @Rule
+       public StreamCollector collector = new StreamCollector();
+
+       @Test
+       public void testCheckpointingWithDelayedAssignment() throws Exception {
+               StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+               env.setParallelism(1);
+               env.setRestartStrategy(new 
RestartStrategies.NoRestartStrategyConfiguration());
+               env.enableCheckpointing(10, CheckpointingMode.EXACTLY_ONCE);
+               final SingleOutputStreamOperator<Long> stream = env
+                       .fromSequence(0, 100)
+                       .map(x -> {
+                               Thread.sleep(10);
+                               return x;
+                       });
+               final CompletableFuture<Collection<Long>> result = 
collector.collect(stream);
+               env.execute();
+
+               assertArrayEquals(LongStream.rangeClosed(0, 
100).boxed().toArray(), result.get().toArray());
+       }
+}

Reply via email to