StefanRRichter commented on a change in pull request #7009: [FLINK-10712]
Support to restore state when using RestartPipelinedRegionStrategy
URL: https://github.com/apache/flink/pull/7009#discussion_r245658407
##########
File path:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
##########
@@ -985,6 +986,67 @@ int getNumScheduledTasks() {
* restore from.
* @param allowNonRestoredState Allow checkpoint state that cannot be
mapped
* to any job vertex in tasks.
+ */
+ public boolean restoreLatestCheckpointedState(
+ Map<JobVertexID, ExecutionJobVertex> tasks,
+ boolean errorIfNoCheckpoint,
+ boolean allowNonRestoredState) throws Exception {
+
+ Map<JobVertexID, BitSet> executionJobVertexIndices = new
HashMap<>(tasks.size());
+ tasks.entrySet().forEach(task -> {
+ ExecutionJobVertex jobVertex = task.getValue();
+ BitSet subTaskIndices = new
BitSet(jobVertex.getParallelism());
+ subTaskIndices.set(0, jobVertex.getParallelism());
+ executionJobVertexIndices.putIfAbsent(task.getKey(),
subTaskIndices);
+ });
+
+ return restoreLatestCheckpointedState(tasks,
executionJobVertexIndices, errorIfNoCheckpoint, allowNonRestoredState);
+ }
+
+ /**
+ * Restores the latest checkpointed state at the granularity of
execution vertex.
+ *
+ * <p> There is no need to check whether checkpoint state could be
mapped to any job vertex
+ * when restoring at the granularity of execution vertex.
+ * @param executionVertices Set of execution vertices to restore. State
for these vertices is
+ * restored via {@link
Execution#setInitialState(JobManagerTaskRestore)}.
+ * @param errorIfNoCheckpoint Fail if no completed checkpoint is
available to
+ * restore from.
+ */
+ public boolean restoreLatestCheckpointedState(
+ List<ExecutionVertex> executionVertices,
+ boolean errorIfNoCheckpoint) throws Exception {
+
+ Map<JobVertexID, ExecutionJobVertex> tasks = new
HashMap<>(executionVertices.size());
+ Map<JobVertexID, BitSet> executionJobVertexIndices = new
HashMap<>(executionVertices.size());
Review comment:
I think it could make more sense to combine the maps `tasks` and
`executionJobVertexIndices`, for example combining them in a `Map<JobVertexID,
ExecutionJobVertexWithSelectedSubtasks>`, where
`ExecutionJobVertexWithSelectedSubtasks` is a pojo that combines
`ExecutionJobVertex` with their corresponding `BitSet` / `Iterable<Integer>`.
This can save lookups and memory for unnecessary data structures.
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
With regards,
Apache Git Services