You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2016/09/08 04:22:21 UTC

[jira] [Commented] (FLINK-4408) Submit Job and setup ExecutionGraph

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

ASF GitHub Bot commented on FLINK-4408:
---------------------------------------

GitHub user KurtYoung opened a pull request:

    https://github.com/apache/flink/pull/2480

    [FLINK-4408][JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph

    Introduce JobMasterRunner to deal with job level leader election and make underlying job manager properly reacted. Also this runner takes care of determining whether job should be submitted with recover fashion. 
    
    This PR also implement the job submission skeleton and setting up the ExecutionGraph, but the interactions with client has been marked as TODO since it may rely on something like JobClientGateway. I'd like to take care of that in a seperate PR later.
    
    The main procedure of managing the lifecycle of a job is:
    
    * Once we received a job submission request from the user, we create a JobMasterRunner to deal with it. 
    * JobMasterRunner will first create leader election service to contend the leader of this job, once leadership is granted, it will try to do some real submission work. 
    * Any error occurred during the submission phase will make this job as rejected and dropped.
    * Once job is accepted, we will face two levels on job retry:
      1. Retarting upon execution failure, it's been taken care of RestartStrategy, and it's mainly __inside__ the ExecutionGraph
      2. Loose of leadership, it will be handled by JobMasterRunner, the old ExecutionGraph will be suspended and then disposed. Retrying should re-submit the job to the JobMaster with isRecovery marked as true. 
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/KurtYoung/flink flink-4408

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/2480.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2480
    
----
commit c5afeb1718f14b47739039b7f4695a791e2f1d20
Author: Kurt Young <yk...@gmail.com>
Date:   2016-09-08T04:00:13Z

    [FLINK-4408][JobManager] Introduce JobMasterRunner and implement job submission & setting up the ExecutionGraph

----


> Submit Job and setup ExecutionGraph
> -----------------------------------
>
>                 Key: FLINK-4408
>                 URL: https://issues.apache.org/jira/browse/FLINK-4408
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Cluster Management
>            Reporter: Xiaogang Shi
>            Assignee: Kurt Young
>
> Once granted the leadership, JM will start to execute the job.
> Most code remains the same except that 
> (1) In old implementation where JM manages the execution of multiple jobs, JM has to load all submitted JobGraphs from SubmittedJobGraphStore and recover them. Now that the components creating JM will be responsible for the recovery of JobGraphs, JM will be created with submitted/recovered JobGraph, without the need to load the JobGraph.
> (2) JM should not rely on Akka to listen on the updates of JobStatus and Execution.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)