diff --git a/docs/dev/stream/state/checkpointing.md b/docs/dev/stream/state/checkpointing.md index eca8d5c1ede95..fab1a494edde9 100644 --- a/docs/dev/stream/state/checkpointing.md +++ b/docs/dev/stream/state/checkpointing.md @@ -76,6 +76,8 @@ Other parameters for checkpointing include: - *fail/continue task on checkpoint errors*: This determines if a task will be failed if an error occurs in the execution of the task's checkpoint procedure. This is the default behaviour. Alternatively, when this is disabled, the task will simply decline the checkpoint to the checkpoint coordinator and continue running. + - *prefer checkpoint for recovery*: This determines if a job will fallback to latest checkpoint even when there are more recent savepoints available to potentially reduce recovery time. +
After a call to this method, {@link #getLatestCheckpoint()} returns the latest + *
After a call to this method, {@link #getLatestCheckpoint(boolean)} returns the latest
* available checkpoint.
*/
void recover() throws Exception;
@@ -47,7 +54,33 @@ public interface CompletedCheckpointStore {
* Returns the latest {@link CompletedCheckpoint} instance or We have a timeout in case the ZooKeeper store get's into a deadlock/livelock situation.
+ */
+ @Test(timeout = 50000)
+ public void testCheckpointRecoveryPreferCheckpoint() throws Exception {
+ final JobID jobID = new JobID();
+ final long checkpoint1Id = 1L;
+ final long checkpoint2Id = 2;
+ final Listnull
if none was
* added.
*/
- CompletedCheckpoint getLatestCheckpoint() throws Exception;
+ default CompletedCheckpoint getLatestCheckpoint(boolean isPreferCheckpointForRecovery) throws Exception {
+ if (getAllCheckpoints().isEmpty()) {
+ return null;
+ }
+
+ CompletedCheckpoint candidate = getAllCheckpoints().get(getAllCheckpoints().size() - 1);
+ if (isPreferCheckpointForRecovery && getAllCheckpoints().size() > 1) {
+ List
- *
@@ -128,7 +128,7 @@ public void testAddCheckpointMoreThanMaxRetained() throws Exception {
public void testEmptyState() throws Exception {
CompletedCheckpointStore checkpoints = createCompletedCheckpoints(1);
- assertNull(checkpoints.getLatestCheckpoint());
+ assertNull(checkpoints.getLatestCheckpoint(false));
assertEquals(0, checkpoints.getAllCheckpoints().size());
assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints());
}
@@ -179,7 +179,7 @@ public void testDiscardAllCheckpoints() throws Exception {
checkpoints.shutdown(JobStatus.FINISHED);
// Empty state
- assertNull(checkpoints.getLatestCheckpoint());
+ assertNull(checkpoints.getLatestCheckpoint(false));
assertEquals(0, checkpoints.getAllCheckpoints().size());
assertEquals(0, checkpoints.getNumberOfRetainedCheckpoints());
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
index f9f50fe6dcb0f..bdb9975fdbc32 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ExecutionGraphCheckpointCoordinatorTest.java
@@ -166,7 +166,8 @@ private ExecutionGraph createExecutionGraphAndEnableCheckpointing(
counter,
store,
new MemoryStateBackend(),
- CheckpointStatsTrackerTest.createTestTracker());
+ CheckpointStatsTrackerTest.createTestTracker(),
+ false);
JobVertex jobVertex = new JobVertex("MockVertex");
jobVertex.setInvokableClass(AbstractInvokable.class);
@@ -222,7 +223,7 @@ public void addCheckpoint(CompletedCheckpoint checkpoint) {
}
@Override
- public CompletedCheckpoint getLatestCheckpoint() {
+ public CompletedCheckpoint getLatestCheckpoint(boolean isPreferCheckpointForRecovery) {
throw new UnsupportedOperationException("Not implemented.");
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
index c0c18773e6f30..af6d860e88a94 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreITCase.java
@@ -119,7 +119,7 @@ public void testRecover() throws Exception {
assertEquals(3, ZOOKEEPER.getClient().getChildren().forPath(CHECKPOINT_PATH).size());
assertEquals(3, checkpoints.getNumberOfRetainedCheckpoints());
- assertEquals(expected[2], checkpoints.getLatestCheckpoint());
+ assertEquals(expected[2], checkpoints.getLatestCheckpoint(false));
Listnull
,null
,true
if a job recovery should fallback to checkpoint.
+ */
+ @PublicEvolving
+ public boolean isPreferCheckpointForRecovery() {
+ return preferCheckpointForRecovery;
+ }
+
+ /**
+ * Sets whether a job recovery should fallback to checkpoint when there is a more recent savepoint.
+ */
+ @PublicEvolving
+ public void setPreferCheckpointForRecovery(boolean preferCheckpointForRecovery) {
+ this.preferCheckpointForRecovery = preferCheckpointForRecovery;
+ }
+
/**
* Returns the cleanup behaviour for externalized checkpoints.
*
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
index 05d06265109d3..fdb38f5f3f323 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java
@@ -700,7 +700,8 @@ private void configureCheckpointing() {
cfg.getMinPauseBetweenCheckpoints(),
cfg.getMaxConcurrentCheckpoints(),
retentionAfterTermination,
- isExactlyOnce),
+ isExactlyOnce,
+ cfg.isPreferCheckpointForRecovery()),
serializedStateBackend,
serializedHooks);
diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java
index f3d904efb3d86..986e4108677a5 100644
--- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java
+++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointIT.java
@@ -244,7 +244,8 @@ private void setUpJobGraph(
10,
1,
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
- true),
+ true,
+ false),
null));
clusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader());
diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java
index 75e9ed18483b4..6635e318ac056 100644
--- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java
@@ -106,7 +106,8 @@ private void setUpWithCheckpointInterval(long checkpointInterval) throws Excepti
10,
1,
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
- true),
+ true,
+ false),
null));
clusterClient.submitJob(jobGraph, ClassLoader.getSystemClassLoader());