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

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

Github user tillrohrmann commented on a diff in the pull request:

    https://github.com/apache/flink/pull/3295#discussion_r101746831
  
    --- Diff: 
flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionGraphUtils.java
 ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.flink.runtime.executiongraph;
    +
    +import org.apache.flink.runtime.concurrent.BiFunction;
    +import org.apache.flink.runtime.concurrent.Future;
    +import org.apache.flink.runtime.instance.SimpleSlot;
    +import org.apache.flink.util.ExceptionUtils;
    +
    +import java.util.List;
    +
    +/**
    + * Utilities for dealing with the execution graphs and scheduling.
    + */
    +public class ExecutionGraphUtils {
    +
    +   /**
    +    * Releases the slot represented by the given future. If the future is 
complete, the
    +    * slot is immediately released. Otherwise, the slot is released as 
soon as the future
    +    * is completed.
    +    * 
    +    * <p>Note that releasing the slot means cancelling any task execution 
currently
    +    * associated with that slot.
    +    * 
    +    * @param slotFuture The future for the slot to release.
    +    */
    +   public static void releaseSlotFuture(Future<SimpleSlot> slotFuture) {
    +           slotFuture.handle(ReleaseSlotFunction.INSTANCE);
    +   }
    +
    +   /**
    +    * Releases the all the slots in the list of arrays of {@code 
ExecutionAndSlot}.
    +    * For each future in that collection holds: If the future is complete, 
its slot is
    +    * immediately released. Otherwise, the slot is released as soon as the 
future
    +    * is completed.
    +    * 
    +    * <p>This methods never throws any exceptions (subclasses of {@code 
Exception})
    +    * and continues to release the remaining slots if one slot release 
failed. We only
    +    * catch Exceptions here (and not other throwables) because the code 
executed while
    +    * releasing slot does not involve any dynamic 
    --- End diff --
    
    sentence incomplete


> 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