nicoweidner commented on a change in pull request #17671:
URL: https://github.com/apache/flink/pull/17671#discussion_r742929075



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java
##########
@@ -141,6 +158,93 @@ public void testAutomaticScaleUp() throws Exception {
         assertTrue(jobResultFuture.join().isSuccess());
     }
 
+    @Test
+    public void testCheckpointStatsPersistedAcrossRescale() throws Exception {
+        final MiniCluster miniCluster = miniClusterResource.getMiniCluster();
+
+        JobVertex jobVertex = new JobVertex("jobVertex", JOB_VERTEX_ID);
+        jobVertex.setInvokableClass(CheckpointingNoOpInvokable.class);
+        jobVertex.setParallelism(PARALLELISM);
+
+        final JobGraph jobGraph = 
JobGraphTestUtils.streamingJobGraph(jobVertex);
+        jobGraph.setSnapshotSettings(
+                new JobCheckpointingSettings(
+                        CheckpointCoordinatorConfiguration.builder()
+                                .setCheckpointInterval(100)
+                                .setCheckpointTimeout(1000)
+                                .build(),
+                        null));
+
+        miniCluster.submitJob(jobGraph).join();
+
+        // wait until some checkpoints have been completed
+        CommonTestUtils.waitUntilCondition(
+                () ->
+                        miniCluster
+                                .getExecutionGraph(jobGraph.getJobID())
+                                .thenApply(
+                                        eg ->
+                                                eg.getCheckpointStatsSnapshot()
+                                                                .getCounts()
+                                                                
.getNumberOfCompletedCheckpoints()
+                                                        > 0)
+                                .get(),
+                Deadline.fromNow(Duration.ofHours(1)));
+
+        miniCluster.terminateTaskManager(0);
+
+        waitUntilParallelismForVertexReached(
+                jobGraph.getJobID(),
+                JOB_VERTEX_ID,
+                NUMBER_SLOTS_PER_TASK_MANAGER * (NUMBER_TASK_MANAGERS - 1));
+
+        // check that the very first checkpoint is still accessible
+        final List<AbstractCheckpointStats> checkpointHistory =
+                miniCluster
+                        .getExecutionGraph(jobGraph.getJobID())
+                        .thenApply(
+                                eg -> 
eg.getCheckpointStatsSnapshot().getHistory().getCheckpoints())
+                        .get();
+        assertThat(checkpointHistory.get(checkpointHistory.size() - 
1).getCheckpointId(), is(1L));

Review comment:
       I was confused by this at first, but I suppose `checkpointHistory` is 
ordered from newest to oldest? (i.e. new checkpoints are added at the beginning 
of the list)

##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveSchedulerClusterITCase.java
##########
@@ -141,6 +158,93 @@ public void testAutomaticScaleUp() throws Exception {
         assertTrue(jobResultFuture.join().isSuccess());
     }
 
+    @Test
+    public void testCheckpointStatsPersistedAcrossRescale() throws Exception {
+        final MiniCluster miniCluster = miniClusterResource.getMiniCluster();
+
+        JobVertex jobVertex = new JobVertex("jobVertex", JOB_VERTEX_ID);
+        jobVertex.setInvokableClass(CheckpointingNoOpInvokable.class);
+        jobVertex.setParallelism(PARALLELISM);
+
+        final JobGraph jobGraph = 
JobGraphTestUtils.streamingJobGraph(jobVertex);
+        jobGraph.setSnapshotSettings(
+                new JobCheckpointingSettings(
+                        CheckpointCoordinatorConfiguration.builder()
+                                .setCheckpointInterval(100)
+                                .setCheckpointTimeout(1000)
+                                .build(),
+                        null));
+
+        miniCluster.submitJob(jobGraph).join();
+
+        // wait until some checkpoints have been completed
+        CommonTestUtils.waitUntilCondition(
+                () ->
+                        miniCluster
+                                .getExecutionGraph(jobGraph.getJobID())
+                                .thenApply(
+                                        eg ->
+                                                eg.getCheckpointStatsSnapshot()
+                                                                .getCounts()
+                                                                
.getNumberOfCompletedCheckpoints()
+                                                        > 0)
+                                .get(),
+                Deadline.fromNow(Duration.ofHours(1)));
+
+        miniCluster.terminateTaskManager(0);
+
+        waitUntilParallelismForVertexReached(
+                jobGraph.getJobID(),
+                JOB_VERTEX_ID,
+                NUMBER_SLOTS_PER_TASK_MANAGER * (NUMBER_TASK_MANAGERS - 1));
+
+        // check that the very first checkpoint is still accessible
+        final List<AbstractCheckpointStats> checkpointHistory =
+                miniCluster
+                        .getExecutionGraph(jobGraph.getJobID())
+                        .thenApply(
+                                eg -> 
eg.getCheckpointStatsSnapshot().getHistory().getCheckpoints())
+                        .get();
+        assertThat(checkpointHistory.get(checkpointHistory.size() - 
1).getCheckpointId(), is(1L));
+    }
+
+    /** An invokable that doesn't do anything interesting, but does support 
checkpointing. */
+    public static class CheckpointingNoOpInvokable extends AbstractInvokable {
+
+        private static final long CANCEL_SIGNAL = -2L;
+        private final BlockingQueue<Long> checkpointsToConfirm = new 
ArrayBlockingQueue<>(1);
+
+        public CheckpointingNoOpInvokable(Environment environment) {
+            super(environment);
+        }
+
+        @Override
+        public void invoke() throws Exception {
+            long signal = -1L;
+            do {
+                signal = checkpointsToConfirm.take();
+                getEnvironment().acknowledgeCheckpoint(signal, new 
CheckpointMetrics());

Review comment:
       Is it intentional that this acknowledges the cancel signal once it is 
put on the queue?

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactory.java
##########
@@ -80,6 +85,13 @@ public DefaultExecutionGraphFactory(
         this.blobWriter = blobWriter;
         this.shuffleMaster = shuffleMaster;
         this.jobMasterPartitionTracker = jobMasterPartitionTracker;
+        this.checkpointStatsTrackerFactory =

Review comment:
       Thoughts on renaming the field `checkpointStatsTrackerSupplier`? It's 
minor, but I intuitively expect a factory to produce a new object when calling 
its production method.
   
   If "factory" is commonly used like this in Flink, feel free to ignore




-- 
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: issues-unsubscr...@flink.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to