expectedCheckpoints = new ArrayList<>(3);
expectedCheckpoints.add(expected[1]);
@@ -219,7 +219,7 @@ public void testSuspendKeepsCheckpoints() throws Exception {
sharedStateRegistry.close();
store = createRecoveredCompletedCheckpointStore(1);
- CompletedCheckpoint recovered = store.getLatestCheckpoint(false);
+ CompletedCheckpoint recovered = store.getLatestCheckpoint();
assertEquals(checkpoint, recovered);
}
@@ -247,7 +247,7 @@ public void testLatestCheckpointRecovery() throws Exception {
sharedStateRegistry.close();
final CompletedCheckpoint latestCheckpoint =
- createRecoveredCompletedCheckpointStore(numCheckpoints).getLatestCheckpoint(false);
+ createRecoveredCompletedCheckpointStore(numCheckpoints).getLatestCheckpoint();
assertEquals(checkpoints.get(checkpoints.size() - 1), latestCheckpoint);
}
@@ -278,7 +278,7 @@ public void testConcurrentCheckpointOperations() throws Exception {
final CompletedCheckpointStore zkCheckpointStore2 =
createRecoveredCompletedCheckpointStore(numberOfCheckpoints);
- CompletedCheckpoint recoveredCheckpoint = zkCheckpointStore2.getLatestCheckpoint(false);
+ CompletedCheckpoint recoveredCheckpoint = zkCheckpointStore2.getLatestCheckpoint();
assertTrue(recoveredCheckpoint instanceof TestCompletedCheckpoint);
TestCompletedCheckpoint recoveredTestCheckpoint =
(TestCompletedCheckpoint) recoveredCheckpoint;
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreMockitoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreMockitoTest.java
index 222f82b10ee25..323c4a9961442 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreMockitoTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/ZooKeeperCompletedCheckpointStoreMockitoTest.java
@@ -175,7 +175,7 @@ public Void answer(InvocationOnMock invocation)
Executors.directExecutor());
CompletedCheckpoint latestCompletedCheckpoint =
- zooKeeperCompletedCheckpointStore.getLatestCheckpoint(false);
+ zooKeeperCompletedCheckpointStore.getLatestCheckpoint();
// check that we return the latest retrievable checkpoint
// this should remove the latest checkpoint because it is broken
@@ -198,140 +198,4 @@ public Void answer(InvocationOnMock invocation)
verify(retrievableStateHandle1, never()).discardState();
verify(retrievableStateHandle2, never()).discardState();
}
-
- /**
- * Tests that the completed checkpoint store can retrieve all checkpoints stored in ZooKeeper
- * and ignores those which cannot be retrieved via their state handles.
- *
- * 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 List, String>>
- checkpointsInZooKeeper = new ArrayList<>(4);
-
- final Collection expectedCheckpointIds = new HashSet<>(2);
- expectedCheckpointIds.add(1L);
- expectedCheckpointIds.add(2L);
-
- final RetrievableStateHandle retrievableStateHandle1 =
- mock(RetrievableStateHandle.class);
- when(retrievableStateHandle1.retrieveState())
- .then(
- (invocation) ->
- new CompletedCheckpoint(
- jobID,
- checkpoint1Id,
- 1L,
- 1L,
- new HashMap<>(),
- null,
- CheckpointProperties.forCheckpoint(
- CheckpointRetentionPolicy
- .NEVER_RETAIN_AFTER_TERMINATION),
- new TestCompletedCheckpointStorageLocation()));
-
- final RetrievableStateHandle retrievableStateHandle2 =
- mock(RetrievableStateHandle.class);
- when(retrievableStateHandle2.retrieveState())
- .then(
- (invocation ->
- new CompletedCheckpoint(
- jobID,
- checkpoint2Id,
- 2L,
- 2L,
- new HashMap<>(),
- null,
- CheckpointProperties.forSavepoint(true),
- new TestCompletedCheckpointStorageLocation())));
-
- checkpointsInZooKeeper.add(Tuple2.of(retrievableStateHandle1, "/foobar1"));
- checkpointsInZooKeeper.add(Tuple2.of(retrievableStateHandle2, "/foobar2"));
-
- final CuratorFramework client = mock(CuratorFramework.class, Mockito.RETURNS_DEEP_STUBS);
- final RetrievableStateStorageHelper storageHelperMock =
- mock(RetrievableStateStorageHelper.class);
-
- ZooKeeperStateHandleStore zooKeeperStateHandleStoreMock =
- spy(new ZooKeeperStateHandleStore<>(client, storageHelperMock));
- doReturn(checkpointsInZooKeeper).when(zooKeeperStateHandleStoreMock).getAllAndLock();
-
- final int numCheckpointsToRetain = 1;
-
- // Mocking for the delete operation on the CuratorFramework client
- // It assures that the callback is executed synchronously
-
- final EnsurePath ensurePathMock = mock(EnsurePath.class);
- final CuratorEvent curatorEventMock = mock(CuratorEvent.class);
- when(curatorEventMock.getType()).thenReturn(CuratorEventType.DELETE);
- when(curatorEventMock.getResultCode()).thenReturn(0);
- when(client.newNamespaceAwareEnsurePath(anyString())).thenReturn(ensurePathMock);
-
- when(client.delete().inBackground(any(BackgroundCallback.class), any(Executor.class)))
- .thenAnswer(
- new Answer>() {
- @Override
- public ErrorListenerPathable answer(InvocationOnMock invocation)
- throws Throwable {
- final BackgroundCallback callback =
- (BackgroundCallback) invocation.getArguments()[0];
-
- ErrorListenerPathable result =
- mock(ErrorListenerPathable.class);
-
- when(result.forPath(anyString()))
- .thenAnswer(
- new Answer() {
- @Override
- public Void answer(InvocationOnMock invocation)
- throws Throwable {
-
- callback.processResult(
- client, curatorEventMock);
-
- return null;
- }
- });
-
- return result;
- }
- });
-
- CompletedCheckpointStore zooKeeperCompletedCheckpointStore =
- new DefaultCompletedCheckpointStore<>(
- numCheckpointsToRetain,
- zooKeeperStateHandleStoreMock,
- zooKeeperCheckpointStoreUtil,
- DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints(
- zooKeeperStateHandleStoreMock, zooKeeperCheckpointStoreUtil),
- Executors.directExecutor());
-
- CompletedCheckpoint latestCompletedCheckpoint =
- zooKeeperCompletedCheckpointStore.getLatestCheckpoint(true);
-
- // check that we return the latest retrievable checkpoint
- // this should remove the latest checkpoint because it is broken
- assertEquals(checkpoint1Id, latestCompletedCheckpoint.getCheckpointID());
-
- // this should remove the second broken checkpoint because we're iterating over all
- // checkpoints
- List completedCheckpoints =
- zooKeeperCompletedCheckpointStore.getAllCheckpoints();
-
- Collection actualCheckpointIds = new HashSet<>(completedCheckpoints.size());
-
- for (CompletedCheckpoint completedCheckpoint : completedCheckpoints) {
- actualCheckpointIds.add(completedCheckpoint.getCheckpointID());
- }
-
- assertEquals(expectedCheckpointIds, actualCheckpointIds);
-
- // check that we did not discard any of the state handles
- verify(retrievableStateHandle1, never()).discardState();
- verify(retrievableStateHandle2, never()).discardState();
- }
}
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
index 4636812c172f7..e751f64ad9d6b 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ArchivedExecutionGraphTest.java
@@ -104,7 +104,6 @@ public static void setupExecutionGraph() throws Exception {
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
true,
false,
- false,
0,
0);
JobCheckpointingSettings checkpointingSettings =
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java
index 1eb39d776150f..a1db269bf6ea2 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/DefaultExecutionGraphDeploymentTest.java
@@ -672,7 +672,6 @@ private ExecutionGraph createExecutionGraph(Configuration configuration) throws
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
false,
false,
- false,
0,
0),
null));
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
index dd4424bf15d16..3caedb0ee7999 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphTest.java
@@ -397,7 +397,6 @@ private static JobCheckpointingSettings createCheckpointSettingsWithInterval(
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
true,
false,
- false,
0,
0);
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java
index 8a7a73804e34d..3a9c2dfce0c80 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/tasks/JobCheckpointingSettingsTest.java
@@ -44,7 +44,6 @@ public void testIsJavaSerializable() throws Exception {
CheckpointRetentionPolicy.RETAIN_ON_FAILURE,
false,
false,
- false,
0,
0),
new SerializedValue<>(new MemoryStateBackend()));
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
index 455e338078264..db070c09e957d 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTest.java
@@ -830,7 +830,7 @@ public void testRestoringFromSavepoint() throws Exception {
// been created
taskSubmitLatch.await();
final CompletedCheckpoint savepointCheckpoint =
- completedCheckpointStore.getLatestCheckpoint(false);
+ completedCheckpointStore.getLatestCheckpoint();
assertThat(savepointCheckpoint, Matchers.notNullValue());
@@ -881,7 +881,7 @@ public void testCheckpointPrecedesSavepointRecovery() throws Exception {
try {
// starting the JobMaster should have read the savepoint
final CompletedCheckpoint savepointCheckpoint =
- completedCheckpointStore.getLatestCheckpoint(false);
+ completedCheckpointStore.getLatestCheckpoint();
assertThat(savepointCheckpoint, Matchers.notNullValue());
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestUtils.java
index ebd0ae3d20d33..b5054b0aeb762 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/jobmaster/TestUtils.java
@@ -103,7 +103,6 @@ public static JobGraph createJobGraphFromJobVerticesWithCheckpointing(
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION)
.setExactlyOnce(true)
.setUnalignedCheckpointsEnabled(false)
- .setPreferCheckpointForRecovery(false)
.setTolerableCheckpointFailureNumber(0)
.build();
final JobCheckpointingSettings checkpointingSettings =
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java
index b755ad94e6363..ae967ceb1cc65 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/DefaultExecutionGraphFactoryTest.java
@@ -107,7 +107,7 @@ public void testRestoringModifiedJobFromSavepointWithAllowNonRestoredStateSuccee
SchedulerBase.computeVertexParallelismStore(jobGraphWithNewOperator),
log);
- final CompletedCheckpoint savepoint = completedCheckpointStore.getLatestCheckpoint(false);
+ final CompletedCheckpoint savepoint = completedCheckpointStore.getLatestCheckpoint();
MatcherAssert.assertThat(savepoint, notNullValue());
diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java
index bf3a9f8db8660..6967c7243adf8 100644
--- a/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java
+++ b/flink-runtime/src/test/java/org/apache/flink/runtime/scheduler/SchedulerTestingUtils.java
@@ -162,7 +162,6 @@ public static void enableCheckpointing(
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
false,
false,
- false,
0,
0);
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
index 7efd3b45ba0b5..8204c6759a790 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java
@@ -118,9 +118,6 @@ public class CheckpointConfig implements java.io.Serializable {
*/
@Deprecated private boolean failOnCheckpointingErrors = true;
- /** Determines if a job will fallback to checkpoint when there is a more recent savepoint. * */
- private boolean preferCheckpointForRecovery = false;
-
/**
* Determines the threshold that we tolerance declined checkpoint failure number. The default
* value is -1 meaning undetermined and not set via {@link
@@ -147,7 +144,6 @@ public CheckpointConfig(final CheckpointConfig checkpointConfig) {
this.checkpointTimeout = checkpointConfig.checkpointTimeout;
this.maxConcurrentCheckpoints = checkpointConfig.maxConcurrentCheckpoints;
this.minPauseBetweenCheckpoints = checkpointConfig.minPauseBetweenCheckpoints;
- this.preferCheckpointForRecovery = checkpointConfig.preferCheckpointForRecovery;
this.tolerableCheckpointFailureNumber = checkpointConfig.tolerableCheckpointFailureNumber;
this.unalignedCheckpointsEnabled = checkpointConfig.isUnalignedCheckpointsEnabled();
this.alignedCheckpointTimeout = checkpointConfig.alignedCheckpointTimeout;
@@ -463,37 +459,6 @@ public boolean isExternalizedCheckpointsEnabled() {
return externalizedCheckpointCleanup != null;
}
- /**
- * Returns whether a job recovery should fallback to checkpoint when there is a more recent
- * savepoint.
- *
- * @return true
if a job recovery should fallback to checkpoint.
- * @deprecated Don't activate prefer checkpoints for recovery because it can lead to data loss
- * and duplicate output. This option will soon be removed. See FLINK-20427 for more
- * information.
- */
- @PublicEvolving
- @Deprecated
- public boolean isPreferCheckpointForRecovery() {
- return preferCheckpointForRecovery;
- }
-
- /**
- * Sets whether a job recovery should fallback to checkpoint when there is a more recent
- * savepoint.
- *
- * @deprecated Don't activate prefer checkpoints for recovery because it can lead to data loss
- * and duplicate output. This option will soon be removed. See FLINK-20427 for more
- * information.
- */
- @PublicEvolving
- @Deprecated
- public void setPreferCheckpointForRecovery(boolean preferCheckpointForRecovery) {
- this.preferCheckpointForRecovery = preferCheckpointForRecovery;
- }
-
/**
* Enables unaligned checkpoints, which greatly reduce checkpointing times under backpressure.
*
@@ -800,9 +765,6 @@ public void configure(ReadableConfig configuration) {
configuration
.getOptional(ExecutionCheckpointingOptions.MIN_PAUSE_BETWEEN_CHECKPOINTS)
.ifPresent(m -> this.setMinPauseBetweenCheckpoints(m.toMillis()));
- configuration
- .getOptional(ExecutionCheckpointingOptions.PREFER_CHECKPOINT_FOR_RECOVERY)
- .ifPresent(this::setPreferCheckpointForRecovery);
configuration
.getOptional(ExecutionCheckpointingOptions.TOLERABLE_FAILURE_NUMBER)
.ifPresent(this::setTolerableCheckpointFailureNumber);
diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
index 27a757476b5c4..f919363826964 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/ExecutionCheckpointingOptions.java
@@ -78,14 +78,6 @@ public class ExecutionCheckpointingOptions {
+ "sure that a minimum amount of time passes where no checkpoint is in progress at all.")
.build());
- public static final ConfigOption PREFER_CHECKPOINT_FOR_RECOVERY =
- ConfigOptions.key("execution.checkpointing.prefer-checkpoint-for-recovery")
- .booleanType()
- .defaultValue(false)
- .withDescription(
- "If enabled, a job recovery should fallback to checkpoint when there is a more recent "
- + "savepoint.");
-
public static final ConfigOption TOLERABLE_FAILURE_NUMBER =
ConfigOptions.key("execution.checkpointing.tolerable-failed-checkpoints")
.intType()
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 f0123e582df9d..536577bc761d1 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
@@ -1315,7 +1315,6 @@ private void configureCheckpointing() {
.setCheckpointRetentionPolicy(retentionAfterTermination)
.setExactlyOnce(
getCheckpointingMode(cfg) == CheckpointingMode.EXACTLY_ONCE)
- .setPreferCheckpointForRecovery(cfg.isPreferCheckpointForRecovery())
.setTolerableCheckpointFailureNumber(
cfg.getTolerableCheckpointFailureNumber())
.setUnalignedCheckpointsEnabled(cfg.isUnalignedCheckpointsEnabled())
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/CheckpointConfigFromConfigurationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/CheckpointConfigFromConfigurationTest.java
index 0bbab9909adcb..6c01193e7247e 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/CheckpointConfigFromConfigurationTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/environment/CheckpointConfigFromConfigurationTest.java
@@ -69,12 +69,6 @@ public static Collection specs() {
.viaSetter(CheckpointConfig::setMinPauseBetweenCheckpoints)
.getterVia(CheckpointConfig::getMinPauseBetweenCheckpoints)
.nonDefaultValue(100L),
- TestSpec.testValue(true)
- .whenSetFromFile(
- "execution.checkpointing.prefer-checkpoint-for-recovery", "true")
- .viaSetter(CheckpointConfig::setPreferCheckpointForRecovery)
- .getterVia(CheckpointConfig::isPreferCheckpointForRecovery)
- .nonDefaultValue(true),
TestSpec.testValue(
CheckpointConfig.ExternalizedCheckpointCleanup
.RETAIN_ON_CANCELLATION)
diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java
index 5a98d140c927d..2164968dca274 100644
--- a/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java
+++ b/flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterStopWithSavepointITCase.java
@@ -289,7 +289,6 @@ private void setUpJobGraph(
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
true,
false,
- false,
0,
0),
null);
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 1361c4ef7ec3e..0e3e7543b6836 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
@@ -107,7 +107,6 @@ private void setUpWithCheckpointInterval(long checkpointInterval) throws Excepti
CheckpointRetentionPolicy.NEVER_RETAIN_AFTER_TERMINATION,
true,
false,
- false,
0,
0),
null);