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

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

zentol closed pull request #5829: [FLINK-9056] [jobmanager] Job submission 
fails with AskTimeoutExcepti…
URL: https://github.com/apache/flink/pull/5829
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
index 64e602f4fbf..83e64b1ea8f 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/Execution.java
@@ -41,6 +41,7 @@
 import org.apache.flink.runtime.io.network.partition.ResultPartitionID;
 import org.apache.flink.runtime.jobmanager.scheduler.CoLocationConstraint;
 import 
org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint;
+import 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
 import org.apache.flink.runtime.jobmanager.scheduler.SlotSharingGroup;
 import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
@@ -409,7 +410,13 @@ public void setInitialState(@Nullable 
JobManagerTaskRestore taskRestore) {
                        // IMPORTANT: We have to use the synchronous handle 
operation (direct executor) here so
                        // that we directly deploy the tasks if the slot 
allocation future is completed. This is
                        // necessary for immediate deployment.
-                       final CompletableFuture<Void> deploymentFuture = 
allocationFuture.handle(
+                       final CompletableFuture<Void> deploymentFuture = 
allocationFuture.exceptionally((Throwable t) -> {
+                               if (t.getCause() instanceof TimeoutException) {
+                                       throw new CompletionException(new 
NoResourceAvailableException("Can't allocated resource for "
+                                               + this.vertex + ", possibly 
there is no more slot available..."));
+                               }
+                               throw new CompletionException(t.getCause());
+                       }).handle(
                                (Execution ignored, Throwable throwable) -> {
                                        if (throwable != null) {
                                                
markFailed(ExceptionUtils.stripCompletionException(throwable));
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
index 51d1827e668..379e931136c 100644
--- 
a/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionVertexSchedulingTest.java
@@ -27,6 +27,7 @@
 import org.apache.flink.runtime.instance.SimpleSlot;
 import org.apache.flink.runtime.jobgraph.JobVertexID;
 import 
org.apache.flink.runtime.jobmanager.scheduler.LocationPreferenceConstraint;
+import 
org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException;
 import org.apache.flink.runtime.jobmanager.scheduler.ScheduledUnit;
 import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
 import org.apache.flink.runtime.jobmanager.slots.ActorTaskManagerGateway;
@@ -37,6 +38,8 @@
 import org.junit.Test;
 
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.TimeoutException;
 
 import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getExecutionVertex;
 import static 
org.apache.flink.runtime.executiongraph.ExecutionGraphTestUtils.getInstance;
@@ -146,4 +149,42 @@ public void testScheduleToDeploying() {
                        fail(e.getMessage());
                }
        }
+
+
+       @Test
+       public void testScheduleToDeployingAndTestTimeOutException() {
+               try {
+                       final ExecutionJobVertex ejv = getExecutionVertex(new 
JobVertexID());
+                       final ExecutionVertex vertex = new ExecutionVertex(ejv, 
0, new IntermediateResult[0],
+                               AkkaUtils.getDefaultTimeout());
+
+                       final Instance instance = getInstance(new 
ActorTaskManagerGateway(
+                               new 
ExecutionGraphTestUtils.SimpleActorGateway(TestingUtils.defaultExecutionContext())));
+                       final SimpleSlot slot = instance.allocateSimpleSlot();
+
+                       Scheduler scheduler = mock(Scheduler.class);
+                       CompletableFuture<LogicalSlot> future = new 
CompletableFuture<>();
+                       future.complete(slot);
+                       when(scheduler.allocateSlot(any(SlotRequestId.class), 
any(ScheduledUnit.class), anyBoolean(), any(SlotProfile.class), 
any(Time.class))).thenReturn(future);
+
+                       assertEquals(ExecutionState.CREATED, 
vertex.getExecutionState());
+
+                       //throw TimeOutException intentially
+
+                       when(scheduler.allocateSlot(any(SlotRequestId.class), 
any(ScheduledUnit.class), anyBoolean(), any(SlotProfile.class), 
any(Time.class))).thenThrow(new CompletionException(new TimeoutException()));
+
+                       vertex.scheduleForExecution(scheduler, false, 
LocationPreferenceConstraint.ALL);
+                       assertEquals(ExecutionState.FAILED, 
vertex.getExecutionState());
+
+                       try {
+                               future.get();
+                       } catch (Throwable e) {
+                               assertTrue(e.getCause() instanceof 
NoResourceAvailableException);
+                       }
+               }
+               catch (Exception e) {
+                       e.printStackTrace();
+                       fail(e.getMessage());
+               }
+       }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


> Job submission fails with AskTimeoutException if not enough slots are 
> available
> -------------------------------------------------------------------------------
>
>                 Key: FLINK-9056
>                 URL: https://issues.apache.org/jira/browse/FLINK-9056
>             Project: Flink
>          Issue Type: Improvement
>          Components: Job-Submission
>    Affects Versions: 1.5.0
>         Environment: * FLIP-6 enabled
>  * Local Flink instance with fixed number of TMs
>  * Job parallelism exceeds available slots
>            Reporter: Fabian Hueske
>            Assignee: yuqi
>            Priority: Major
>
> The error message if a job submission fails due to lack of available slots is 
> not helpful:
> {code:java}
> Caused by: akka.pattern.AskTimeoutException: Ask timed out on 
> [Actor[akka://flink/user/8f0fabba-4021-45b6-a1f7-b8afd6627640#-574617182|#-574617182]]
>  after [300000 ms]. Sender[null] sent message of type 
> "org.apache.flink.runtime.rpc.messages.LocalRpcInvocation".
>      at 
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>      at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>      at 
> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>      at 
> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>      at 
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>      at 
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>      at 
> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>      at 
> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>      at 
> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>      at java.lang.Thread.run(Thread.java:748)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to