[
https://issues.apache.org/jira/browse/FLINK-8360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16348501#comment-16348501
]
ASF GitHub Bot commented on FLINK-8360:
---------------------------------------
Github user tillrohrmann commented on a diff in the pull request:
https://github.com/apache/flink/pull/5239#discussion_r165341071
--- Diff:
flink-runtime/src/main/java/org/apache/flink/runtime/taskexecutor/TaskManagerServices.java
---
@@ -197,7 +201,13 @@ public static TaskManagerServices fromConfiguration(
final JobManagerTable jobManagerTable = new JobManagerTable();
final JobLeaderService jobLeaderService = new
JobLeaderService(taskManagerLocation);
- final TaskExecutorLocalStateStoresManager taskStateManager =
new TaskExecutorLocalStateStoresManager();
+
+ final File taskExecutorLocalStateRootDir =
+ new
File(taskManagerServicesConfiguration.getLocalStateRootDir(),
LOCAL_STATE_SUB_DIRECTORY_ROOT);
--- End diff --
Are we giving the `TaskExecutorLocalStateStoresManager` a unique directory?
What happens if multiple TMs run on the same machine? Will they use different
directories?
> Implement task-local state recovery
> -----------------------------------
>
> Key: FLINK-8360
> URL: https://issues.apache.org/jira/browse/FLINK-8360
> Project: Flink
> Issue Type: New Feature
> Components: State Backends, Checkpointing
> Reporter: Stefan Richter
> Assignee: Stefan Richter
> Priority: Major
> Fix For: 1.5.0
>
>
> This issue tracks the development of recovery from task-local state. The main
> idea is to have a secondary, local copy of the checkpointed state, while
> there is still a primary copy in DFS that we report to the checkpoint
> coordinator.
> Recovery can attempt to restore from the secondary local copy, if available,
> to save network bandwidth. This requires that the assignment from tasks to
> slots is as sticky is possible.
> For starters, we will implement this feature for all managed keyed states and
> can easily enhance it to all other state types (e.g. operator state) later.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)