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

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_r101766916
  
    --- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/concurrent/FutureUtilsTest.java
 ---
    @@ -0,0 +1,190 @@
    +/*
    + * 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.concurrent;
    +
    +import org.apache.flink.runtime.concurrent.FutureUtils.ConjunctFuture;
    +import org.apache.flink.runtime.concurrent.impl.FlinkCompletableFuture;
    +
    +import org.junit.Test;
    +
    +import java.io.IOException;
    +import java.util.Arrays;
    +import java.util.Collections;
    +import java.util.concurrent.ExecutionException;
    +
    +import static org.junit.Assert.*;
    +
    +/**
    + * Tests for the utility methods in {@link FutureUtils}
    + */
    +public class FutureUtilsTest {
    +
    +   @Test
    +   public void testConjunctFutureFailsOnEmptyAndNull() throws Exception {
    +           try {
    +                   FutureUtils.combineAll(null);
    +                   fail();
    +           } catch (NullPointerException ignored) {}
    +
    +           try {
    +                   
FutureUtils.combineAll(Collections.<Future<?>>emptyList());
    +                   fail();
    +           } catch (IllegalArgumentException ignored) {}
    +
    +           try {
    +                   FutureUtils.combineAll(Arrays.asList(
    +                                   new FlinkCompletableFuture<Object>(),
    +                                   null,
    +                                   new FlinkCompletableFuture<Object>()));
    +                   fail();
    +           } catch (NullPointerException ignored) {}
    +   }
    +
    +   @Test
    +   public void testConjunctFutureCompletion() throws Exception {
    +           // some futures that we combine
    +           CompletableFuture<Object> future1 = new 
FlinkCompletableFuture<>();
    +           CompletableFuture<Object> future2 = new 
FlinkCompletableFuture<>();
    +           CompletableFuture<Object> future3 = new 
FlinkCompletableFuture<>();
    +           CompletableFuture<Object> future4 = new 
FlinkCompletableFuture<>();
    +
    +           // some future is initially completed
    +           future2.complete(new Object());
    +
    +           // build the conjunct future
    +           ConjunctFuture result = 
FutureUtils.combineAll(Arrays.asList(future1, future2, future3, future4));
    +
    +           Future<Void> resultMapped = result.thenAccept(new 
AcceptFunction<Void>() {
    +                   @Override
    +                   public void accept(Void value) {}
    +           });
    --- End diff --
    
    ah makes sense :-)


> 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