[ https://issues.apache.org/jira/browse/FLINK-16018?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17039228#comment-17039228 ]
Till Rohrmann commented on FLINK-16018: --------------------------------------- I agree that {{TimeoutExceptions}} should not surface to the user. They are hard to understand and cryptic. I also agree that the root problem is the blocking operation which we should run in a separate thread(pool). Apart from these two aspects, I actually agree with Andrey that we have a problem of un-aligned timeouts here. The {{AskTimeoutException}} the user sees is caused by the {{JobSubmitHandler}} which waits on the {{DisptacherGateway.submitJob}} with a timeout of 10s (default value). The {{JobManager}} creation depends on the Amazon SDK client which, I assume, does some retries and delays in between. Since the 10s are smaller than the overall SDK client timeout, we only see the {{AskTimeoutException}}. This is also underlined by Robert's post where he set {{web.timeout}} to 30s and then sees the actual cause of failure. Retrying the operation in case of a timeout is at the moment not an option since the submit operation is not idempotent. Also, {{DispatcherGateway.submitJob}} currently requires the creation of the {{ExecutionGraph}} before returning. One idea could be to not wait for the creation before acknowledging the job submission. One could define the job submission as that all files have been uploaded to the cluster. This is actually something I wanted to change with FLINK-11719. Another thought could be to say that we require the {{Dispatcher}} to always respond eventually. If this assumption holds, then we could set the {{web.timeout}} to infinity. We do this at other places where we now that the actors run in the same actor system as well. As long as the rest server and the cluster entrypoint run in the same JVM, this assumption might be ok. Of course, the consequence of such a change would be that any kind of (dead/live)locks in downstream components would propagate through to the {{RestServer}}. Maybe setting {{web.timeout}} to 10 minutes would then be a compromise. > Improve error reporting when submitting batch job (instead of > AskTimeoutException) > ---------------------------------------------------------------------------------- > > Key: FLINK-16018 > URL: https://issues.apache.org/jira/browse/FLINK-16018 > Project: Flink > Issue Type: Improvement > Components: Runtime / Coordination > Affects Versions: 1.11.0 > Reporter: Robert Metzger > Priority: Major > > While debugging the {{Shaded Hadoop S3A end-to-end test (minio)}} pre-commit > test, I noticed that the JobSubmission is not producing very helpful error > messages. > Environment: > - A simple batch wordcount job > - a unavailable minio s3 filesystem service > What happens from a user's perspective: > - The job submission fails after 10 seconds with a AskTimeoutException: > {code} > 2020-02-07T11:38:27.1189393Z akka.pattern.AskTimeoutException: Ask timed out > on [Actor[akka://flink/user/dispatcher#-939201095]] after [10000 ms]. Message > of type [org.apache.flink.runtime.rpc.messages.LocalFencedMessage]. A typical > reason for `AskTimeoutException` is that the recipient actor didn't send a > reply. > 2020-02-07T11:38:27.1189538Z at > akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635) > 2020-02-07T11:38:27.1189616Z at > akka.pattern.PromiseActorRef$$anonfun$2.apply(AskSupport.scala:635) > 2020-02-07T11:38:27.1189713Z at > akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:648) > 2020-02-07T11:38:27.1189789Z at > akka.actor.Scheduler$$anon$4.run(Scheduler.scala:205) > 2020-02-07T11:38:27.1189883Z at > scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601) > 2020-02-07T11:38:27.1189973Z at > scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109) > 2020-02-07T11:38:27.1190067Z at > scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599) > 2020-02-07T11:38:27.1190159Z at > akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:328) > 2020-02-07T11:38:27.1190267Z at > akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:279) > 2020-02-07T11:38:27.1190358Z at > akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:283) > 2020-02-07T11:38:27.1190465Z at > akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:235) > 2020-02-07T11:38:27.1190540Z at java.lang.Thread.run(Thread.java:748) > {code} > What a user would expect: > - An error message indicating why the job submission failed. -- This message was sent by Atlassian Jira (v8.3.4#803005)