jiangxin369 commented on code in PR #22670:
URL: https://github.com/apache/flink/pull/22670#discussion_r1245015259


##########
flink-tests/src/test/java/org/apache/flink/test/checkpointing/ImmediateCheckpointingAfterAllTasksFinishedITCase.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.test.checkpointing;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.api.common.typeinfo.Types;
+import org.apache.flink.api.connector.sink2.Sink;
+import org.apache.flink.api.connector.sink2.SinkWriter;
+import org.apache.flink.core.execution.SavepointFormatType;
+import org.apache.flink.runtime.jobgraph.JobGraph;
+import org.apache.flink.runtime.jobgraph.JobVertex;
+import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.runtime.minicluster.MiniClusterConfiguration;
+import org.apache.flink.runtime.state.StateSnapshotContext;
+import org.apache.flink.runtime.testutils.CommonTestUtils;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.streaming.api.operators.AbstractStreamOperator;
+import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
+import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.junit.Assert;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.io.TempDir;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+/**
+ * Tests an immediate checkpoint should be triggered right after all tasks 
reached the end of data.
+ */
+public class ImmediateCheckpointingAfterAllTasksFinishedITCase extends 
AbstractTestBase {
+    private static final int SMALL_SOURCE_NUM_RECORDS = 20;
+    private static final int BIG_SOURCE_NUM_RECORDS = 100;
+
+    private StreamExecutionEnvironment env;
+
+    @TempDir private java.nio.file.Path tmpDir;
+
+    @BeforeEach
+    public void setUp() {
+        env = StreamExecutionEnvironment.getExecutionEnvironment();
+        env.setRestartStrategy(RestartStrategies.noRestart());
+        env.enableCheckpointing(Long.MAX_VALUE - 1);
+    }
+
+    @ParameterizedTest
+    @ValueSource(ints = {1, 4})
+    public void testImmediateCheckpointing(int parallelism) throws Exception {

Review Comment:
   The test can not pass because the checkpoint interval is set to `Long.MAX - 
1` so the test would keep waiting for the final checkpoint which is also the 
first, and would never finish.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/StateWithExecutionGraph.java:
##########
@@ -195,6 +200,18 @@ void declineCheckpoint(DeclineCheckpoint decline) {
         executionGraphHandler.declineCheckpoint(decline);
     }
 
+    void notifyEndOfData(ExecutionAttemptID executionAttemptID) {
+        CheckpointCoordinatorConfiguration checkpointCoordinatorConfiguration =
+                executionGraph.getCheckpointCoordinatorConfiguration();
+        if (checkpointCoordinatorConfiguration != null

Review Comment:
   - The adaptive scheduler only supports streaming mode refer to the 
[doc](https://github.com/apache/flink/blob/e35f33f539b09b2c18fb826a144c08541e148dd0/flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java#L152C68-L152C68),
 we don't have to check job type.
   - CheckpiontCoordination and CheckpointCoordinatorConfiguration are only 
initialized when enable checkpointing, so NotNull it is equivalent to 
checkpoint enabled. Here is a [Similar 
usage](https://github.com/apache/flink/blob/e35f33f539b09b2c18fb826a144c08541e148dd0/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/checkpoints/CheckpointConfigHandler.java#L100C40-L100C40).



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