Skip to content

Commit f64261c

Browse files
dmvktillrohrmann
authored andcommitted
[FLINK-22483][runtime] Remove CompletedCheckpointStore#recover) method an change contract of CheckpointRecoveryFactory#createCompletedCheckpointStore, so that newly constructed CheckpointStore is already recovered.
It's enough to recover CompletedCheckpointStoreshould only once, right after JobMasterRunner gains leadership. This also ensures that we'll fetch checkpoints from the external store in a "jobmaster-future-thread", without pontetially blocking RPC threads. This closes apache#16652.
1 parent 94ae3dc commit f64261c

File tree

43 files changed

+809
-652
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

43 files changed

+809
-652
lines changed

flink-kubernetes/src/main/java/org/apache/flink/kubernetes/highavailability/KubernetesCheckpointRecoveryFactory.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ public KubernetesCheckpointRecoveryFactory(
7070
}
7171

7272
@Override
73-
public CompletedCheckpointStore createCheckpointStore(
73+
public CompletedCheckpointStore createRecoveredCompletedCheckpointStore(
7474
JobID jobID, int maxNumberOfCheckpointsToRetain, ClassLoader userClassLoader)
7575
throws Exception {
7676

flink-kubernetes/src/main/java/org/apache/flink/kubernetes/utils/KubernetesUtils.java

+3
Original file line numberDiff line numberDiff line change
@@ -33,6 +33,7 @@
3333
import org.apache.flink.runtime.checkpoint.CompletedCheckpoint;
3434
import org.apache.flink.runtime.checkpoint.CompletedCheckpointStore;
3535
import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStore;
36+
import org.apache.flink.runtime.checkpoint.DefaultCompletedCheckpointStoreUtils;
3637
import org.apache.flink.runtime.highavailability.HighAvailabilityServicesUtils;
3738
import org.apache.flink.runtime.jobgraph.JobGraph;
3839
import org.apache.flink.runtime.jobmanager.DefaultJobGraphStore;
@@ -299,6 +300,8 @@ public static CompletedCheckpointStore createCompletedCheckpointStore(
299300
maxNumberOfCheckpointsToRetain,
300301
stateHandleStore,
301302
KubernetesCheckpointStoreUtil.INSTANCE,
303+
DefaultCompletedCheckpointStoreUtils.retrieveCompletedCheckpoints(
304+
stateHandleStore, KubernetesCheckpointStoreUtil.INSTANCE),
302305
executor);
303306
}
304307

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

+2-7
Original file line numberDiff line numberDiff line change
@@ -1521,16 +1521,11 @@ private OptionalLong restoreLatestCheckpointedStateInternal(
15211521
}
15221522

15231523
// We create a new shared state registry object, so that all pending async disposal
1524-
// requests from previous
1525-
// runs will go against the old object (were they can do no harm).
1526-
// This must happen under the checkpoint lock.
1524+
// requests from previous runs will go against the old object (were they can do no
1525+
// harm). This must happen under the checkpoint lock.
15271526
sharedStateRegistry.close();
15281527
sharedStateRegistry = sharedStateRegistryFactory.create(executor);
15291528

1530-
// Recover the checkpoints, TODO this could be done only when there is a new leader, not
1531-
// on each recovery
1532-
completedCheckpointStore.recover();
1533-
15341529
// Now, we re-register all (shared) states from the checkpoint store with the new
15351530
// registry
15361531
for (CompletedCheckpoint completedCheckpoint :

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

+4-2
Original file line numberDiff line numberDiff line change
@@ -24,14 +24,16 @@
2424
public interface CheckpointRecoveryFactory {
2525

2626
/**
27-
* Creates a {@link CompletedCheckpointStore} instance for a job.
27+
* Creates a RECOVERED {@link CompletedCheckpointStore} instance for a job. In this context,
28+
* RECOVERED means, that if we already have completed checkpoints from previous runs, we should
29+
* use them as the initial state.
2830
*
2931
* @param jobId Job ID to recover checkpoints for
3032
* @param maxNumberOfCheckpointsToRetain Maximum number of checkpoints to retain
3133
* @param userClassLoader User code class loader of the job
3234
* @return {@link CompletedCheckpointStore} instance for the job
3335
*/
34-
CompletedCheckpointStore createCheckpointStore(
36+
CompletedCheckpointStore createRecoveredCompletedCheckpointStore(
3537
JobID jobId, int maxNumberOfCheckpointsToRetain, ClassLoader userClassLoader)
3638
throws Exception;
3739

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

+1-9
Original file line numberDiff line numberDiff line change
@@ -32,14 +32,6 @@ public interface CompletedCheckpointStore {
3232

3333
Logger LOG = LoggerFactory.getLogger(CompletedCheckpointStore.class);
3434

35-
/**
36-
* Recover available {@link CompletedCheckpoint} instances.
37-
*
38-
* <p>After a call to this method, {@link #getLatestCheckpoint(boolean)} returns the latest
39-
* available checkpoint.
40-
*/
41-
void recover() throws Exception;
42-
4335
/**
4436
* Adds a {@link CompletedCheckpoint} instance to the list of completed checkpoints.
4537
*
@@ -107,7 +99,7 @@ default long getLatestCheckpointId() {
10799
* or kept.
108100
*
109101
* @param jobStatus Job state on shut down
110-
* @param checkpointsCleaner that will cleanup copmpleted checkpoints if needed
102+
* @param checkpointsCleaner that will cleanup completed checkpoints if needed
111103
*/
112104
void shutdown(JobStatus jobStatus, CheckpointsCleaner checkpointsCleaner) throws Exception;
113105

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

-5
Original file line numberDiff line numberDiff line change
@@ -30,11 +30,6 @@
3030
public enum DeactivatedCheckpointCompletedCheckpointStore implements CompletedCheckpointStore {
3131
INSTANCE;
3232

33-
@Override
34-
public void recover() throws Exception {
35-
throw unsupportedOperationException();
36-
}
37-
3833
@Override
3934
public void addCheckpoint(
4035
CompletedCheckpoint checkpoint,

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

+31-132
Original file line numberDiff line numberDiff line change
@@ -19,24 +19,20 @@
1919
package org.apache.flink.runtime.checkpoint;
2020

2121
import org.apache.flink.api.common.JobStatus;
22-
import org.apache.flink.api.java.tuple.Tuple2;
2322
import org.apache.flink.runtime.persistence.PossibleInconsistentStateException;
2423
import org.apache.flink.runtime.persistence.ResourceVersion;
2524
import org.apache.flink.runtime.persistence.StateHandleStore;
26-
import org.apache.flink.runtime.state.RetrievableStateHandle;
27-
import org.apache.flink.util.FlinkException;
25+
import org.apache.flink.util.Preconditions;
2826

2927
import org.slf4j.Logger;
3028
import org.slf4j.LoggerFactory;
3129

32-
import java.io.IOException;
3330
import java.util.ArrayDeque;
3431
import java.util.ArrayList;
35-
import java.util.Comparator;
32+
import java.util.Collection;
3633
import java.util.List;
37-
import java.util.Set;
3834
import java.util.concurrent.Executor;
39-
import java.util.stream.Collectors;
35+
import java.util.concurrent.atomic.AtomicBoolean;
4036

4137
import static org.apache.flink.util.Preconditions.checkArgument;
4238
import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -61,9 +57,6 @@ public class DefaultCompletedCheckpointStore<R extends ResourceVersion<R>>
6157
private static final Logger LOG =
6258
LoggerFactory.getLogger(DefaultCompletedCheckpointStore.class);
6359

64-
private static final Comparator<Tuple2<RetrievableStateHandle<CompletedCheckpoint>, String>>
65-
STRING_COMPARATOR = Comparator.comparing(o -> o.f1);
66-
6760
/** Completed checkpoints state handle store. */
6861
private final StateHandleStore<CompletedCheckpoint, R> checkpointStateHandleStore;
6962

@@ -81,6 +74,9 @@ public class DefaultCompletedCheckpointStore<R extends ResourceVersion<R>>
8174

8275
private final CheckpointStoreUtil completedCheckpointStoreUtil;
8376

77+
/** False if store has been shutdown. */
78+
private final AtomicBoolean running = new AtomicBoolean(true);
79+
8480
/**
8581
* Creates a {@link DefaultCompletedCheckpointStore} instance.
8682
*
@@ -95,18 +91,14 @@ public DefaultCompletedCheckpointStore(
9591
int maxNumberOfCheckpointsToRetain,
9692
StateHandleStore<CompletedCheckpoint, R> stateHandleStore,
9793
CheckpointStoreUtil completedCheckpointStoreUtil,
94+
Collection<CompletedCheckpoint> completedCheckpoints,
9895
Executor executor) {
99-
10096
checkArgument(maxNumberOfCheckpointsToRetain >= 1, "Must retain at least one checkpoint.");
101-
10297
this.maxNumberOfCheckpointsToRetain = maxNumberOfCheckpointsToRetain;
103-
10498
this.checkpointStateHandleStore = checkNotNull(stateHandleStore);
105-
10699
this.completedCheckpoints = new ArrayDeque<>(maxNumberOfCheckpointsToRetain + 1);
107-
100+
this.completedCheckpoints.addAll(completedCheckpoints);
108101
this.ioExecutor = checkNotNull(executor);
109-
110102
this.completedCheckpointStoreUtil = checkNotNull(completedCheckpointStoreUtil);
111103
}
112104

@@ -115,55 +107,13 @@ public boolean requiresExternalizedCheckpoints() {
115107
return true;
116108
}
117109

118-
/**
119-
* Recover all the valid checkpoints from state handle store. All the successfully recovered
120-
* checkpoints will be added to {@link #completedCheckpoints} sorted by checkpoint id.
121-
*/
122-
@Override
123-
public void recover() throws Exception {
124-
LOG.info("Recovering checkpoints from {}.", checkpointStateHandleStore);
125-
126-
// Get all there is first
127-
final List<Tuple2<RetrievableStateHandle<CompletedCheckpoint>, String>> initialCheckpoints =
128-
checkpointStateHandleStore.getAllAndLock();
129-
130-
initialCheckpoints.sort(STRING_COMPARATOR);
131-
132-
final int numberOfInitialCheckpoints = initialCheckpoints.size();
133-
134-
LOG.info(
135-
"Found {} checkpoints in {}.",
136-
numberOfInitialCheckpoints,
137-
checkpointStateHandleStore);
138-
if (haveAllDownloaded(initialCheckpoints)) {
139-
LOG.info(
140-
"All {} checkpoints found are already downloaded.", numberOfInitialCheckpoints);
141-
return;
142-
}
143-
144-
final List<CompletedCheckpoint> retrievedCheckpoints =
145-
new ArrayList<>(numberOfInitialCheckpoints);
146-
LOG.info("Trying to fetch {} checkpoints from storage.", numberOfInitialCheckpoints);
147-
148-
for (Tuple2<RetrievableStateHandle<CompletedCheckpoint>, String> checkpointStateHandle :
149-
initialCheckpoints) {
150-
retrievedCheckpoints.add(
151-
checkNotNull(retrieveCompletedCheckpoint(checkpointStateHandle)));
152-
}
153-
154-
// Clear local handles in order to prevent duplicates on recovery. The local handles should
155-
// reflect the state handle store contents.
156-
completedCheckpoints.clear();
157-
completedCheckpoints.addAll(retrievedCheckpoints);
158-
}
159-
160110
/**
161111
* Synchronously writes the new checkpoints to state handle store and asynchronously removes
162112
* older ones.
163113
*
164114
* @param checkpoint Completed checkpoint to add.
165115
* @throws PossibleInconsistentStateException if adding the checkpoint failed and leaving the
166-
* system in an possibly inconsistent state, i.e. it's uncertain whether the checkpoint
116+
* system in a possibly inconsistent state, i.e. it's uncertain whether the checkpoint
167117
* metadata was fully written to the underlying systems or not.
168118
*/
169119
@Override
@@ -172,13 +122,13 @@ public void addCheckpoint(
172122
CheckpointsCleaner checkpointsCleaner,
173123
Runnable postCleanup)
174124
throws Exception {
175-
125+
Preconditions.checkState(running.get(), "Checkpoint store has already been shutdown.");
176126
checkNotNull(checkpoint, "Checkpoint");
177127

178128
final String path =
179129
completedCheckpointStoreUtil.checkpointIDToName(checkpoint.getCheckpointID());
180130

181-
// Now add the new one. If it fails, we don't want to loose existing data.
131+
// Now add the new one. If it fails, we don't want to lose existing data.
182132
checkpointStateHandleStore.addAndLock(path, checkpoint);
183133

184134
completedCheckpoints.addLast(checkpoint);
@@ -214,30 +164,28 @@ public int getMaxNumberOfRetainedCheckpoints() {
214164
@Override
215165
public void shutdown(JobStatus jobStatus, CheckpointsCleaner checkpointsCleaner)
216166
throws Exception {
217-
if (jobStatus.isGloballyTerminalState()) {
218-
LOG.info("Shutting down");
219-
220-
for (CompletedCheckpoint checkpoint : completedCheckpoints) {
221-
try {
222-
tryRemoveCompletedCheckpoint(
223-
checkpoint,
224-
checkpoint.shouldBeDiscardedOnShutdown(jobStatus),
225-
checkpointsCleaner,
226-
() -> {});
227-
} catch (Exception e) {
228-
LOG.warn("Fail to remove checkpoint during shutdown.", e);
167+
if (running.compareAndSet(true, false)) {
168+
if (jobStatus.isGloballyTerminalState()) {
169+
LOG.info("Shutting down");
170+
for (CompletedCheckpoint checkpoint : completedCheckpoints) {
171+
try {
172+
tryRemoveCompletedCheckpoint(
173+
checkpoint,
174+
checkpoint.shouldBeDiscardedOnShutdown(jobStatus),
175+
checkpointsCleaner,
176+
() -> {});
177+
} catch (Exception e) {
178+
LOG.warn("Fail to remove checkpoint during shutdown.", e);
179+
}
229180
}
181+
completedCheckpoints.clear();
182+
checkpointStateHandleStore.clearEntries();
183+
} else {
184+
LOG.info("Suspending");
185+
// Clear the local handles, but don't remove any state
186+
completedCheckpoints.clear();
187+
checkpointStateHandleStore.releaseAll();
230188
}
231-
232-
completedCheckpoints.clear();
233-
checkpointStateHandleStore.clearEntries();
234-
} else {
235-
LOG.info("Suspending");
236-
237-
// Clear the local handles, but don't remove any state
238-
completedCheckpoints.clear();
239-
240-
checkpointStateHandleStore.releaseAll();
241189
}
242190
}
243191

@@ -257,25 +205,6 @@ private void tryRemoveCompletedCheckpoint(
257205
}
258206
}
259207

260-
private boolean haveAllDownloaded(
261-
List<Tuple2<RetrievableStateHandle<CompletedCheckpoint>, String>> checkpointPointers) {
262-
if (completedCheckpoints.size() != checkpointPointers.size()) {
263-
return false;
264-
}
265-
Set<Long> localIds =
266-
completedCheckpoints.stream()
267-
.map(CompletedCheckpoint::getCheckpointID)
268-
.collect(Collectors.toSet());
269-
for (Tuple2<RetrievableStateHandle<CompletedCheckpoint>, String> initialCheckpoint :
270-
checkpointPointers) {
271-
if (!localIds.contains(
272-
completedCheckpointStoreUtil.nameToCheckpointID(initialCheckpoint.f1))) {
273-
return false;
274-
}
275-
}
276-
return true;
277-
}
278-
279208
/**
280209
* Tries to remove the checkpoint identified by the given checkpoint id.
281210
*
@@ -286,34 +215,4 @@ private boolean tryRemove(long checkpointId) throws Exception {
286215
return checkpointStateHandleStore.releaseAndTryRemove(
287216
completedCheckpointStoreUtil.checkpointIDToName(checkpointId));
288217
}
289-
290-
private CompletedCheckpoint retrieveCompletedCheckpoint(
291-
Tuple2<RetrievableStateHandle<CompletedCheckpoint>, String> stateHandle)
292-
throws FlinkException {
293-
long checkpointId = completedCheckpointStoreUtil.nameToCheckpointID(stateHandle.f1);
294-
295-
LOG.info("Trying to retrieve checkpoint {}.", checkpointId);
296-
297-
try {
298-
return stateHandle.f0.retrieveState();
299-
} catch (ClassNotFoundException cnfe) {
300-
throw new FlinkException(
301-
"Could not retrieve checkpoint "
302-
+ checkpointId
303-
+ " from state handle under "
304-
+ stateHandle.f1
305-
+ ". This indicates that you are trying to recover from state written by an "
306-
+ "older Flink version which is not compatible. Try cleaning the state handle store.",
307-
cnfe);
308-
} catch (IOException ioe) {
309-
throw new FlinkException(
310-
"Could not retrieve checkpoint "
311-
+ checkpointId
312-
+ " from state handle under "
313-
+ stateHandle.f1
314-
+ ". This indicates that the retrieved state handle is broken. Try cleaning the "
315-
+ "state handle store.",
316-
ioe);
317-
}
318-
}
319218
}

0 commit comments

Comments
 (0)