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

Till Rohrmann commented on FLINK-11143:
---------------------------------------

Flink has different timeout values for its different components. For the 
{{MiniCluster}}, the configuration option {{akka.ask.timeout}} is respected. 
For timeouts originating from the REST handlers, one needs to specify 
{{web.timeout}}.

If you encounter a timeout problem, then I would suggest to increase the 
respective timeout configuration option. This should be an easy workaround.

The problem you are seeing is most likely caused by the job submission which 
can be blocking. When submitting a job, the system tries to create the 
{{JobManager}} responsible for running the submitted job. This operation can 
involve accessing a file system and, hence, can block. The proper solution 
would be to make the job submission non-blocking.

I will close this issue in favour of FLINK-16018 because they seem to describe 
the same problem.

> AskTimeoutException is thrown during job submission and completion
> ------------------------------------------------------------------
>
>                 Key: FLINK-11143
>                 URL: https://issues.apache.org/jira/browse/FLINK-11143
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 1.6.2, 1.10.0
>            Reporter: Alex Vinnik
>            Priority: Critical
>
> For more details please see the thread
> [http://mail-archives.apache.org/mod_mbox/flink-user/201812.mbox/%[email protected]%3E]
> On submission 
> 2018-12-12 02:28:31 ERROR JobsOverviewHandler:92 - Implementation error: 
> Unhandled exception.
>  akka.pattern.AskTimeoutException: Ask timed out on 
> [Actor[akka://flink/user/dispatcher#225683351|#225683351]] after [10000 ms]. 
> Sender[null] sent message of type 
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>  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)
>  
> On completion
>  
> {"errors":["Internal server error.","<Exception on server 
> side:\njava.util.concurrent.CompletionException: 
> akka.pattern.AskTimeoutException: Ask timed out on 
> [Actor[akka://flink/user/dispatcher#105638574]] after [10000 ms]. 
> Sender[null] sent message of type 
> \"org.apache.flink.runtime.rpc.messages.LocalFencedMessage\".
> at 
> java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
> at 
> java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
> at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
> at 
> java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
> at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
> at 
> java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
> at 
> org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:772)
> at akka.dispatch.OnComplete.internal(Future.scala:258)
> at akka.dispatch.OnComplete.internal(Future.scala:256)
> at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
> at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
> at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:36)
> at 
> org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
> at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:44)
> at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:252)
> at akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:603)
> 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)\nCaused by: 
> akka.pattern.AskTimeoutException: Ask timed out on 
> [Actor[akka://flink/user/dispatcher#105638574]] after [10000 ms]. 
> Sender[null] sent message of type 
> \"org.apache.flink.runtime.rpc.messages.LocalFencedMessage\".
> at 
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)\n\t...
>  9 more\n\nEnd of exception on server side>"]}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to