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-5703) ExecutionGraph recovery based on reconciliation with TaskManager reports
Date Fri, 26 May 2017 08:20:04 GMT

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

ASF GitHub Bot commented on FLINK-5703:

Github user zhijiangW commented on the issue:

    @StephanEwen , sorry for delay update of this feature.
    This modification covers almost all the work on job manager side concerning about the
suggestions you mentioned before, including:
    1. Once `JobManagerRunner` granting leadership, it will check the job status by `RunningJobsRegistry`.
If the job is already running, then the `JobManager` will enter reconciling process and confirm
leadership last.
    To do so, we can confirm that the `TaskManager` will not report task status before `JobManager`
reconciles, and it is also reasonable for 'JobManager` exposing its address to outside `TaskManager`
after internal initialization. To keep the behavior consistency, I modified the `EmbeddedLeaderService`
to set leader flag as true in the process of `grantLeadership` instead of 'confirmLeadership`.
    2. 'ExecutionGraph' recovery based on task status report form `TaskManager`, and I created
the new execution attempt to replace the current one for recovery, also the same case for
`IntermediateResultPartition` to avoid updating the final fields directly in previous way.
And also integrating two phases of recovery into one phase you mentioned before.
    3. After duration time, the `ExecutionGraph` decides to enter `RUNNING` or `FAILING` from
`RECONCILING` based on reconciliation results.
    Currently I have not considered the improvements for `LAZY_FROM_SOURCES` schedule mode
in this version, because it needs to remember how many vertexes have already been scheduled
before `JobManager` failure. Maybe we can improve it in next version if needed.
    I may supplement some extra tests and update the attached design doc after you approve
this realization, and the 'TaskManager' part will be submitted in another separate PR which
mainly includes:
    1. `TaskManager` will not fail task directly after notified of `JobManager` leader lost,
and triggers a timer to later check whether reconnects to the new job leader successfully.
    2. If reconnection success, it will report the current internal task and slot status that
belongs to this job leader, otherwise it will fail the tasks as the current behavior.
    Wish your further response, and I will begin to consider resource issues of FLIP-6, thank

> ExecutionGraph recovery based on reconciliation with TaskManager reports
> ------------------------------------------------------------------------
>                 Key: FLINK-5703
>                 URL: https://issues.apache.org/jira/browse/FLINK-5703
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Distributed Coordination, JobManager
>            Reporter: zhijiang
>            Assignee: zhijiang
> The ExecutionGraph structure would be recovered from TaskManager reports during reconciling
period, and the necessary information includes:
>     - Execution: ExecutionAttemptID, AttemptNumber, StartTimestamp, ExecutionState, SimpleSlot,
PartialInputChannelDeploymentDescriptor(Consumer Execution)
>     - ExecutionVertex: Map<IntermediateResultPartitionID, IntermediateResultPartition>
>     - ExecutionGraph: ConcurrentHashMap<ExecutionAttemptID, Execution>
> For {{RECONCILING}} ExecutionState, it should be transition into any existing task states
({{RUNNING}},{{CANCELED}},{{FAILED}},{{FINISHED}}). To do so, the TaskManger should maintain
the terminal task state ({{CANCELED}},{{FAILED}},{{FINISHED}}) for a while and we try to realize
this mechanism in another jira. In addition, the state transition would trigger different
actions, and some actions rely on above necessary information. Considering this limit, the
recovery process will be divided into two steps:
>     - First, recovery all other necessary information except ExecutionState.
>     - Second, transition ExecutionState into real task state and trigger actions. The
behavior is the same with current {{UpdateTaskExecutorState}}.
> To make logic easy and consistency, during recovery period, all the other RPC messages
({{UpdateTaskExecutionState}}, {{ScheduleOrUpdateConsumers}},etc) from TaskManager should
be refused temporarily and responded with a special message by JobMaster. Then the TaskManager
should retry to send these messages later until JobManager ends recovery and acknowledgement.
> For {{RECONCILING}} JobStatus, it would be transition into one of the states ({{RUNNING}},{{FAILING}},{{FINISHED}})
after recovery.
>     - {{RECONCILING}} to {{RUNNING}}: All the TaskManager report within duration time
and all the tasks are in {{RUNNING}} states.
>     - {{RECONCILING}} to {{FAILING}}: One of the TaskManager does not report in time,
or one of the tasks state is in {{FAILED}} or {{CANCELED}}
>     - {{RECONCILING}} to {{FINISHED}}: All the TaskManger report within duration time
and all the tasks are in {{FINISHED}} states.

This message was sent by Atlassian JIRA

View raw message