[
https://issues.apache.org/jira/browse/FLINK-5892?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15988898#comment-15988898
]
ASF GitHub Bot commented on FLINK-5892:
---------------------------------------
Github user StefanRRichter commented on a diff in the pull request:
https://github.com/apache/flink/pull/3770#discussion_r113920542
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/CheckpointCoordinator.java
---
@@ -1025,11 +1026,11 @@ public boolean restoreLatestCheckpointedState(
LOG.info("Restoring from latest valid checkpoint: {}.",
latest);
// re-assign the task states
-
- final Map<JobVertexID, TaskState> taskStates =
latest.getTaskStates();
+ final Map<OperatorID, OperatorState> operatorStates =
latest.getOperatorStates();
StateAssignmentOperation stateAssignmentOperation =
- new StateAssignmentOperation(LOG,
tasks, taskStates, allowNonRestoredState);
+ new StateAssignmentOperation(LOG,
tasks, operatorStates, allowNonRestoredState);
--- End diff --
Not sure why this is implemented in a way that a logger is passed to the
`StateAssignmentOperation`. I guess the class should simply have its own
logger. I think this could be changed. But seems like this was introduced
earlier and is unrelated to this PR. But I wouldn't to have this refactored to
the normal logger scheme before we merge.
> Recover job state at the granularity of operator
> ------------------------------------------------
>
> Key: FLINK-5892
> URL: https://issues.apache.org/jira/browse/FLINK-5892
> Project: Flink
> Issue Type: New Feature
> Components: State Backends, Checkpointing
> Reporter: Guowei Ma
> Assignee: Guowei Ma
>
> JobGraph has no `Operator` info so `ExecutionGraph` can only recovery at the
> granularity of task.
> This leads to the result that the operator of the job may not recover the
> state from a save point even if the save point has the state of operator.
>
> https://docs.google.com/document/d/19suTRF0nh7pRgeMnIEIdJ2Fq-CcNVt5_hR3cAoICf7Q/edit#.
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)