Skip to content

Commit 3db714d

Browse files
committed
Extracts CheckpointStatsTracker interface and introduces DefaultCheckpointStatsTracker
1 parent f52650d commit 3db714d

19 files changed

+819
-626
lines changed

flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointStatsTracker.java

Lines changed: 26 additions & 533 deletions
Large diffs are not rendered by default.

flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/DefaultCheckpointStatsTracker.java

Lines changed: 571 additions & 0 deletions
Large diffs are not rendered by default.
Lines changed: 91 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,91 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.flink.runtime.checkpoint;
20+
21+
import org.apache.flink.runtime.executiongraph.ExecutionAttemptID;
22+
import org.apache.flink.runtime.jobgraph.JobVertexID;
23+
24+
import java.util.Collections;
25+
import java.util.Map;
26+
import java.util.Set;
27+
28+
public enum NoOpCheckpointStatsTracker implements CheckpointStatsTracker {
29+
INSTANCE;
30+
31+
@Override
32+
public void reportRestoredCheckpoint(
33+
long checkpointID,
34+
CheckpointProperties properties,
35+
String externalPath,
36+
long stateSize) {}
37+
38+
@Override
39+
public void reportCompletedCheckpoint(CompletedCheckpointStats completed) {}
40+
41+
@Override
42+
public PendingCheckpointStats getPendingCheckpointStats(long checkpointId) {
43+
return createPendingCheckpoint(
44+
checkpointId,
45+
System.currentTimeMillis(),
46+
CheckpointProperties.forCheckpoint(
47+
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION),
48+
Collections.emptyMap());
49+
}
50+
51+
@Override
52+
public void reportIncompleteStats(
53+
long checkpointId, ExecutionAttemptID attemptId, CheckpointMetrics metrics) {}
54+
55+
@Override
56+
public void reportInitializationStarted(
57+
Set<ExecutionAttemptID> toInitialize, long initializationStartTs) {}
58+
59+
@Override
60+
public void reportInitializationMetrics(
61+
ExecutionAttemptID executionAttemptId,
62+
SubTaskInitializationMetrics initializationMetrics) {}
63+
64+
@Override
65+
public PendingCheckpointStats reportPendingCheckpoint(
66+
long checkpointId,
67+
long triggerTimestamp,
68+
CheckpointProperties props,
69+
Map<JobVertexID, Integer> vertexToDop) {
70+
return createPendingCheckpoint(checkpointId, triggerTimestamp, props, vertexToDop);
71+
}
72+
73+
private PendingCheckpointStats createPendingCheckpoint(
74+
long checkpointId,
75+
long triggerTimestamp,
76+
CheckpointProperties props,
77+
Map<JobVertexID, Integer> vertexToDop) {
78+
return new PendingCheckpointStats(checkpointId, triggerTimestamp, props, vertexToDop);
79+
}
80+
81+
@Override
82+
public void reportFailedCheckpoint(FailedCheckpointStats failed) {}
83+
84+
@Override
85+
public void reportFailedCheckpointsWithoutInProgress() {}
86+
87+
@Override
88+
public CheckpointStatsSnapshot createSnapshot() {
89+
return CheckpointStatsSnapshot.empty();
90+
}
91+
}

flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphBuilder.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -52,7 +52,6 @@
5252
import org.apache.flink.runtime.state.StateBackendLoader;
5353
import org.apache.flink.util.DynamicCodeLoadingException;
5454
import org.apache.flink.util.SerializedValue;
55-
import org.apache.flink.util.function.CachingSupplier;
5655

5756
import org.slf4j.Logger;
5857

@@ -92,7 +91,7 @@ public static DefaultExecutionGraph buildGraph(
9291
long initializationTimestamp,
9392
VertexAttemptNumberStore vertexAttemptNumberStore,
9493
VertexParallelismStore vertexParallelismStore,
95-
CachingSupplier<CheckpointStatsTracker> checkpointStatsTrackerCachingSupplier,
94+
CheckpointStatsTracker checkpointStatsTracker,
9695
boolean isDynamicGraph,
9796
ExecutionJobVertex.Factory executionJobVertexFactory,
9897
MarkPartitionFinishedStrategy markPartitionFinishedStrategy,
@@ -342,7 +341,7 @@ public static DefaultExecutionGraph buildGraph(
342341
completedCheckpointStore,
343342
rootBackend,
344343
rootStorage,
345-
checkpointStatsTrackerCachingSupplier.get(),
344+
checkpointStatsTracker,
346345
checkpointsCleaner,
347346
jobManagerConfig.get(STATE_CHANGE_LOG_STORAGE));
348347
}

flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactory.java

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,6 @@
4141
import org.apache.flink.runtime.jobmaster.ExecutionDeploymentTrackerDeploymentListenerAdapter;
4242
import org.apache.flink.runtime.metrics.groups.JobManagerJobMetricGroup;
4343
import org.apache.flink.runtime.shuffle.ShuffleMaster;
44-
import org.apache.flink.util.function.CachingSupplier;
4544

4645
import org.slf4j.Logger;
4746

@@ -131,7 +130,7 @@ public ExecutionGraph createAndRestoreExecutionGraph(
131130
CompletedCheckpointStore completedCheckpointStore,
132131
CheckpointsCleaner checkpointsCleaner,
133132
CheckpointIDCounter checkpointIdCounter,
134-
CachingSupplier<CheckpointStatsTracker> checkpointStatsTrackerCachingSupplier,
133+
CheckpointStatsTracker checkpointStatsTracker,
135134
TaskDeploymentDescriptorFactory.PartitionLocationConstraint partitionLocationConstraint,
136135
long initializationTimestamp,
137136
VertexAttemptNumberStore vertexAttemptNumberStore,
@@ -171,7 +170,7 @@ public ExecutionGraph createAndRestoreExecutionGraph(
171170
initializationTimestamp,
172171
vertexAttemptNumberStore,
173172
vertexParallelismStore,
174-
checkpointStatsTrackerCachingSupplier,
173+
checkpointStatsTracker,
175174
isDynamicGraph,
176175
executionJobVertexFactory,
177176
markPartitionFinishedStrategy,

flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionGraphFactory.java

Lines changed: 3 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,6 @@
2828
import org.apache.flink.runtime.executiongraph.MarkPartitionFinishedStrategy;
2929
import org.apache.flink.runtime.executiongraph.VertexAttemptNumberStore;
3030
import org.apache.flink.runtime.jobgraph.JobGraph;
31-
import org.apache.flink.util.function.CachingSupplier;
3231

3332
import org.slf4j.Logger;
3433

@@ -42,10 +41,8 @@ public interface ExecutionGraphFactory {
4241
* @param completedCheckpointStore completedCheckpointStore to pass to the CheckpointCoordinator
4342
* @param checkpointsCleaner checkpointsCleaner to pass to the CheckpointCoordinator
4443
* @param checkpointIdCounter checkpointIdCounter to pass to the CheckpointCoordinator
45-
* @param checkpointStatsTrackerCachingSupplier The {@link CachingSupplier} that is used provide
46-
* the {@link CheckpointStatsTracker}. {@code CachingSupplier} is used here to allow for
47-
* lazy instantiation. This is required to avoid the side effects that appear during the
48-
* creation of a {@code CheckpointStatsTracker} if checkpointing is disabled.
44+
* @param checkpointStatsTracker The {@link CheckpointStatsTracker} that's used for collecting
45+
* the checkpoint-related statistics.
4946
* @param partitionLocationConstraint partitionLocationConstraint for this job
5047
* @param initializationTimestamp initializationTimestamp when the ExecutionGraph was created
5148
* @param vertexAttemptNumberStore vertexAttemptNumberStore keeping information about the vertex
@@ -63,7 +60,7 @@ ExecutionGraph createAndRestoreExecutionGraph(
6360
CompletedCheckpointStore completedCheckpointStore,
6461
CheckpointsCleaner checkpointsCleaner,
6562
CheckpointIDCounter checkpointIdCounter,
66-
CachingSupplier<CheckpointStatsTracker> checkpointStatsTrackerCachingSupplier,
63+
CheckpointStatsTracker checkpointStatsTracker,
6764
TaskDeploymentDescriptorFactory.PartitionLocationConstraint partitionLocationConstraint,
6865
long initializationTimestamp,
6966
VertexAttemptNumberStore vertexAttemptNumberStore,

flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java

Lines changed: 12 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@
4747
import org.apache.flink.runtime.checkpoint.CheckpointsCleaner;
4848
import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
4949
import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
50+
import org.apache.flink.runtime.checkpoint.DefaultCheckpointStatsTracker;
5051
import org.apache.flink.runtime.checkpoint.SubTaskInitializationMetrics;
5152
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
5253
import org.apache.flink.runtime.concurrent.ComponentMainThreadExecutor;
@@ -105,7 +106,6 @@
105106
import org.apache.flink.util.FlinkException;
106107
import org.apache.flink.util.IterableUtils;
107108
import org.apache.flink.util.concurrent.FutureUtils;
108-
import org.apache.flink.util.function.CachingSupplier;
109109

110110
import org.slf4j.Logger;
111111

@@ -223,12 +223,20 @@ public SchedulerBase(
223223
this.deploymentStateTimeMetrics =
224224
new DeploymentStateTimeMetrics(jobGraph.getJobType(), jobStatusMetricsSettings);
225225

226+
final CheckpointStatsTracker checkpointStatsTracker =
227+
SchedulerUtils.createCheckpointStatsTrackerIfCheckpointingIsEnabled(
228+
jobGraph.getCheckpointingSettings(),
229+
() ->
230+
new DefaultCheckpointStatsTracker(
231+
jobMasterConfiguration.get(
232+
WebOptions.CHECKPOINTS_HISTORY_SIZE),
233+
jobManagerJobMetricGroup));
226234
this.executionGraph =
227235
createAndRestoreExecutionGraph(
228236
completedCheckpointStore,
229237
checkpointsCleaner,
230238
checkpointIdCounter,
231-
jobMasterConfiguration.get(WebOptions.CHECKPOINTS_HISTORY_SIZE),
239+
checkpointStatsTracker,
232240
initializationTimestamp,
233241
mainThreadExecutor,
234242
jobStatusListener,
@@ -375,7 +383,7 @@ private ExecutionGraph createAndRestoreExecutionGraph(
375383
CompletedCheckpointStore completedCheckpointStore,
376384
CheckpointsCleaner checkpointsCleaner,
377385
CheckpointIDCounter checkpointIdCounter,
378-
int checkpointsHistorySize,
386+
CheckpointStatsTracker checkpointStatsTracker,
379387
long initializationTimestamp,
380388
ComponentMainThreadExecutor mainThreadExecutor,
381389
JobStatusListener jobStatusListener,
@@ -388,10 +396,7 @@ private ExecutionGraph createAndRestoreExecutionGraph(
388396
completedCheckpointStore,
389397
checkpointsCleaner,
390398
checkpointIdCounter,
391-
new CachingSupplier<>(
392-
() ->
393-
new CheckpointStatsTracker(
394-
checkpointsHistorySize, jobManagerJobMetricGroup)),
399+
checkpointStatsTracker,
395400
TaskDeploymentDescriptorFactory.PartitionLocationConstraint.fromJobType(
396401
jobGraph.getJobType()),
397402
initializationTimestamp,

flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerUtils.java

Lines changed: 19 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,18 +24,24 @@
2424
import org.apache.flink.core.execution.RestoreMode;
2525
import org.apache.flink.runtime.checkpoint.CheckpointIDCounter;
2626
import org.apache.flink.runtime.checkpoint.CheckpointRecoveryFactory;
27+
import org.apache.flink.runtime.checkpoint.CheckpointStatsTracker;
2728
import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
2829
import org.apache.flink.runtime.checkpoint.DeactivatedCheckpointCompletedCheckpointStore;
2930
import org.apache.flink.runtime.checkpoint.DeactivatedCheckpointIDCounter;
3031
import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils;
32+
import org.apache.flink.runtime.checkpoint.NoOpCheckpointStatsTracker;
3133
import org.apache.flink.runtime.client.JobExecutionException;
3234
import org.apache.flink.runtime.executiongraph.DefaultExecutionGraphBuilder;
3335
import org.apache.flink.runtime.jobgraph.JobGraph;
36+
import org.apache.flink.runtime.jobgraph.tasks.JobCheckpointingSettings;
3437
import org.apache.flink.runtime.state.SharedStateRegistry;
3538

3639
import org.slf4j.Logger;
3740

41+
import javax.annotation.Nullable;
42+
3843
import java.util.concurrent.Executor;
44+
import java.util.function.Supplier;
3945

4046
/** Utils class for Flink's scheduler implementations. */
4147
public final class SchedulerUtils {
@@ -107,6 +113,19 @@ public static CheckpointIDCounter createCheckpointIDCounterIfCheckpointingIsEnab
107113
}
108114
}
109115

116+
public static CheckpointStatsTracker createCheckpointStatsTrackerIfCheckpointingIsEnabled(
117+
@Nullable JobCheckpointingSettings checkpointingSettings,
118+
Supplier<CheckpointStatsTracker> checkpointStatsTrackerFactory) {
119+
if (checkpointingSettings != null
120+
&& checkpointingSettings
121+
.getCheckpointCoordinatorConfiguration()
122+
.isCheckpointingEnabled()) {
123+
return checkpointStatsTrackerFactory.get();
124+
} else {
125+
return NoOpCheckpointStatsTracker.INSTANCE;
126+
}
127+
}
128+
110129
private static CheckpointIDCounter createCheckpointIdCounter(
111130
CheckpointRecoveryFactory recoveryFactory, JobID jobId) throws Exception {
112131
return recoveryFactory.createCheckpointIDCounter(jobId);

flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/adaptive/AdaptiveScheduler.java

Lines changed: 7 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,7 @@
4747
import org.apache.flink.runtime.checkpoint.CheckpointsCleaner;
4848
import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
4949
import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
50+
import org.apache.flink.runtime.checkpoint.DefaultCheckpointStatsTracker;
5051
import org.apache.flink.runtime.checkpoint.SubTaskInitializationMetrics;
5152
import org.apache.flink.runtime.checkpoint.TaskStateSnapshot;
5253
import org.apache.flink.runtime.client.JobExecutionException;
@@ -121,7 +122,6 @@
121122
import org.apache.flink.util.FlinkException;
122123
import org.apache.flink.util.Preconditions;
123124
import org.apache.flink.util.concurrent.FutureUtils;
124-
import org.apache.flink.util.function.CachingSupplier;
125125
import org.apache.flink.util.function.FunctionWithException;
126126
import org.apache.flink.util.function.ThrowingConsumer;
127127

@@ -330,7 +330,7 @@ public Duration getMaximumDelayForTriggeringRescale() {
330330
private final CheckpointsCleaner checkpointsCleaner;
331331
private final CompletedCheckpointStore completedCheckpointStore;
332332
private final CheckpointIDCounter checkpointIdCounter;
333-
private final CachingSupplier<CheckpointStatsTracker> checkpointStatsTrackerCachingSupplier;
333+
private final CheckpointStatsTracker checkpointStatsTracker;
334334

335335
private final CompletableFuture<JobStatus> jobTerminationFuture = new CompletableFuture<>();
336336

@@ -422,10 +422,11 @@ public AdaptiveScheduler(
422422
this.checkpointIdCounter =
423423
SchedulerUtils.createCheckpointIDCounterIfCheckpointingIsEnabled(
424424
jobGraph, checkpointRecoveryFactory);
425-
this.checkpointStatsTrackerCachingSupplier =
426-
new CachingSupplier<>(
425+
this.checkpointStatsTracker =
426+
SchedulerUtils.createCheckpointStatsTrackerIfCheckpointingIsEnabled(
427+
jobGraph.getCheckpointingSettings(),
427428
() ->
428-
new CheckpointStatsTracker(
429+
new DefaultCheckpointStatsTracker(
429430
configuration.get(WebOptions.CHECKPOINTS_HISTORY_SIZE),
430431
jobManagerJobMetricGroup));
431432

@@ -1302,7 +1303,7 @@ private ExecutionGraph createExecutionGraphAndRestoreState(
13021303
completedCheckpointStore,
13031304
checkpointsCleaner,
13041305
checkpointIdCounter,
1305-
checkpointStatsTrackerCachingSupplier,
1306+
checkpointStatsTracker,
13061307
TaskDeploymentDescriptorFactory.PartitionLocationConstraint.MUST_BE_KNOWN,
13071308
initializationTimestamp,
13081309
vertexAttemptNumberStore,

flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorFailureTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -212,7 +212,7 @@ private void testStoringFailureHandling(Exception failure, int expectedCleanupCa
212212
new FailingCompletedCheckpointStore(failure);
213213

214214
CheckpointStatsTracker statsTracker =
215-
new CheckpointStatsTracker(
215+
new DefaultCheckpointStatsTracker(
216216
Integer.MAX_VALUE,
217217
UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup());
218218
final AtomicInteger cleanupCallCount = new AtomicInteger(0);

flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinatorMasterHooksTest.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -500,7 +500,7 @@ private CheckpointCoordinator instantiateCheckpointCoordinator(
500500
new ExecutionGraphCheckpointPlanCalculatorContext(graph),
501501
graph.getVerticesTopologically(),
502502
false),
503-
new CheckpointStatsTracker(
503+
new DefaultCheckpointStatsTracker(
504504
1, UnregisteredMetricGroups.createUnregisteredJobManagerJobMetricGroup()));
505505
}
506506

0 commit comments

Comments
 (0)