flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (FLINK-8360) Implement task-local state recovery
Date Thu, 01 Feb 2018 12:34:00 GMT

    [ https://issues.apache.org/jira/browse/FLINK-8360?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16348514#comment-16348514
] 

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_r165343021
  
    --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/LocalStateForwardingTest.java
---
    @@ -156,30 +157,38 @@ public void acknowledgeCheckpoint(
     			}
     		};
     
    -		TaskLocalStateStore taskLocalStateStore = new TaskLocalStateStore(jobID, jobVertexID,
subtaskIdx) {
    -			@Override
    -			public void storeLocalState(
    -				@Nonnull CheckpointMetaData checkpointMetaData,
    -				@Nullable TaskStateSnapshot localState) {
    -
    -				Assert.assertEquals(tm, localState);
    -				tmReported.set(true);
    -			}
    -		};
    +		TemporaryFolder temporaryFolder = new TemporaryFolder();
     
    -		TaskStateManagerImpl taskStateManager =
    -			new TaskStateManagerImpl(
    -				jobID,
    -				executionAttemptID,
    -				taskLocalStateStore,
    -				null,
    -				checkpointResponder);
    -
    -		taskStateManager.reportTaskStateSnapshots(
    -			checkpointMetaData,
    -			checkpointMetrics,
    -			jm,
    -			tm);
    +		try {
    +			TaskLocalStateStore taskLocalStateStore =
    +				new TaskLocalStateStore(jobID, jobVertexID, subtaskIdx, temporaryFolder.newFolder())
{
    +					@Override
    +					public void storeLocalState(
    +						@Nonnull CheckpointMetaData checkpointMetaData,
    +						@Nullable TaskStateSnapshot localState) {
    +
    +						Assert.assertEquals(tm, localState);
    +						tmReported.set(true);
    +					}
    +				};
    +
    +			TaskStateManagerImpl taskStateManager =
    +				new TaskStateManagerImpl(
    +					jobID,
    +					executionAttemptID,
    +					taskLocalStateStore,
    +					null,
    +					checkpointResponder);
    +
    +			taskStateManager.reportTaskStateSnapshots(
    +				checkpointMetaData,
    +				checkpointMetrics,
    +				jm,
    +				tm);
    +		} catch (Exception ex) {
    +			temporaryFolder.delete();
    +			throw new RuntimeException(ex);
    --- End diff --
    
    Why do we throw a `RuntimeException`?


> 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)

Mime
View raw message