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

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

GitHub user StephanEwen opened a pull request:

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

    [FLINK-5747] [distributed coordination] Eager scheduling allocates slots 
and deploys tasks in bulk

    ## Problem Addressed
    
    Currently, eager scheduling immediately triggers the scheduling for all 
vertices and their subtasks in topological order.
    
    This has two problems:
    
      - This works only, as long as resource acquisition is "synchronous". With 
dynamic resource acquisition in FLIP-6, the resources are returned as Futures 
which may complete out of order. This results in out-of-order (not in 
topological order) scheduling of tasks which does not work for streaming.
    
      - Deploying some tasks that depend on other tasks before it is clear that 
the other tasks have resources as well leads to situations where many 
deploy/recovery cycles happen before enough resources are available to get the 
job running fully.
    
    ## Implemented Change
    
      - The `Execution` has separate methods to allocate a resource and to 
deploy the task to that resource
      - The **eager** scheduling mode allocates all resources in one chunk and 
then deploys once all resources are available.
    
    As a utility, this implements the `FutureUtils.combineAll` method that 
combines the Futures of the individual resources to a combined Future.
    
    ## Tests
    
    The main tests are in `ExecutionGraphSchedulingTest`. The used utilities 
are tested in `FutureUtilsTest` and in `ExecutionGraphUtilsTest`.

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

    $ git pull https://github.com/StephanEwen/incubator-flink slot_scheduling

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

    https://github.com/apache/flink/pull/3295.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 #3295
    
----
commit 1f18cbb0d6d119fa5e5c4803201c28887b90cef5
Author: Stephan Ewen <[email protected]>
Date:   2017-02-03T19:26:23Z

    [FLINK-5747] [distributed coordination] Eager scheduling allocates slots 
and deploys tasks in bulk
    
    That way, strictly topological deployment can be guaranteed.
    
    Also, many quick deploy/not-enough-resources/fail/recover cycles can be
    avoided in the cases where resources need some time to appear.

----


> Eager Scheduling should deploy all Tasks together
> -------------------------------------------------
>
>                 Key: FLINK-5747
>                 URL: https://issues.apache.org/jira/browse/FLINK-5747
>             Project: Flink
>          Issue Type: Bug
>          Components: JobManager
>    Affects Versions: 1.2.0
>            Reporter: Stephan Ewen
>            Assignee: Stephan Ewen
>             Fix For: 1.3.0
>
>
> Currently, eager scheduling immediately triggers the scheduling for all 
> vertices and their subtasks in topological order. 
> This has two problems:
>   - This works only, as long as resource acquisition is "synchronous". With 
> dynamic resource acquisition in FLIP-6, the resources are returned as Futures 
> which may complete out of order. This results in out-of-order (not in 
> topological order) scheduling of tasks which does not work for streaming.
>   - Deploying some tasks that depend on other tasks before it is clear that 
> the other tasks have resources as well leads to situations where many 
> deploy/recovery cycles happen before enough resources are available to get 
> the job running fully.
> For eager scheduling, we should allocate all resources in one chunk and then 
> deploy once we know that all are available.
> As a follow-up, the same should be done per pipelined component in lazy batch 
> scheduling as well. That way we get lazy scheduling across blocking 
> boundaries, and bulk (gang) scheduling in pipelined subgroups.
> This also does not apply for efforts of fine grained recovery, where 
> individual tasks request replacement resources.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to