[ https://issues.apache.org/jira/browse/FLINK-8360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16365751#comment-16365751 ]
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_r168500772 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/TaskExecutorLocalStateStoresManager.java --- @@ -59,58 +71,55 @@ public TaskExecutorLocalStateStoresManager(File[] localStateRootDirectories) { } } } - } - public TaskLocalStateStore localStateStoreForTask( - JobID jobId, - JobVertexID jobVertexID, - int subtaskIndex) { + } - Preconditions.checkNotNull(jobId); - final JobVertexSubtaskKey taskKey = new JobVertexSubtaskKey(jobVertexID, subtaskIndex); + public TaskLocalStateStore localStateStoreForSubtask( + @Nonnull JobID jobId, + @Nonnull AllocationID allocationID, + @Nonnull JobVertexID jobVertexID, + @Nonnegative int subtaskIndex) { final Map<JobVertexSubtaskKey, TaskLocalStateStore> taskStateManagers = - this.taskStateStoresByJobID.computeIfAbsent(jobId, k -> new HashMap<>()); + this.taskStateStoresByAllocationID.computeIfAbsent(allocationID, k -> new ConcurrentHashMap<>()); + + final JobVertexSubtaskKey taskKey = new JobVertexSubtaskKey(jobVertexID, subtaskIndex); return taskStateManagers.computeIfAbsent( - taskKey, k -> new TaskLocalStateStore(jobId, jobVertexID, subtaskIndex, localStateRootDirectories)); + taskKey, + k -> new TaskLocalStateStore(jobId, jobVertexID, subtaskIndex, localStateRootDirectories, discardExecutor)); } - public void releaseJob(JobID jobID) { + public void releaseLocalStateForAllocationId(@Nonnull AllocationID allocationID) { - Map<JobVertexSubtaskKey, TaskLocalStateStore> cleanupLocalStores = taskStateStoresByJobID.remove(jobID); + Map<JobVertexSubtaskKey, TaskLocalStateStore> cleanupLocalStores = + taskStateStoresByAllocationID.remove(allocationID); if (cleanupLocalStores != null) { -// doRelease(cleanupLocalStores.values()); + doRelease(cleanupLocalStores.values()); } } public void releaseAll() { - for (Map<JobVertexSubtaskKey, TaskLocalStateStore> stateStoreMap : taskStateStoresByJobID.values()) { -// doRelease(stateStoreMap.values()); + for (Map<JobVertexSubtaskKey, TaskLocalStateStore> stateStoreMap : taskStateStoresByAllocationID.values()) { + doRelease(stateStoreMap.values()); } - taskStateStoresByJobID.clear(); + taskStateStoresByAllocationID.clear(); } - private void doRelease(Iterable<TaskLocalStateStore> toRelease) throws Exception { + private void doRelease(Iterable<TaskLocalStateStore> toRelease) { if (toRelease != null) { - Exception collectedExceptions = null; - for (TaskLocalStateStore stateStore : toRelease) { try { stateStore.dispose(); } catch (Exception disposeEx) { - collectedExceptions = ExceptionUtils.firstOrSuppressed(disposeEx, collectedExceptions); + LOG.warn("Exception while disposing local state store", disposeEx); --- End diff -- I think it would be interesting to know which LocalStateStore could not be disposed. > 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)