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

Yingjie Cao commented on FLINK-13489:
-------------------------------------

I ran the heavy deployment test for many times and found the akka timeout 
problem should be purely resulted by the test environment. The resources of the 
test VM is limited (two cpu cores and less than 8G memory, both the network and 
disk are not fast enough), but the test needs to start 10 TM and more than 100 
tasks. When the akka timeout config value is decreased, the akka timeout 
failures occurred more frequently, and when the akka timeout config value was 
set to 60s, akka timeout failure did not occur after the heavy deployment test 
ran for more than 200 times (if the default timeout is used, there will be one 
or two times of failure when the heavy deployment test ran for 100 times).

There are two config options relevant to the akka timeout, one is 
akka.ask.timeout and most rpc calls use this, the other is web.timeout and the 
rpc calls from rest client use this. The default value for both config options 
are 10s and in my tests, both can cause the akka timeout failure. (To be 
honest, I think we should use the same akka rpc timeout configuration for both 
rest client and other components like TM, JM and RM, when the a user encounters 
the akka time out problem, he/she is easy to find the akka.ask.timeout config 
options and increase it but it is not easy to establish the relationship with 
web.timeout ).

I have create a pr which increases the config value of both  akka.ask.timeout 
and web.timeout, which should make the heavy deployment test more stable (other 
tests are not affected).

> Heavy deployment end-to-end test fails on Travis with TM heartbeat timeout
> --------------------------------------------------------------------------
>
>                 Key: FLINK-13489
>                 URL: https://issues.apache.org/jira/browse/FLINK-13489
>             Project: Flink
>          Issue Type: Bug
>          Components: Test Infrastructure
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Yingjie Cao
>            Priority: Blocker
>              Labels: pull-request-available
>             Fix For: 1.9.0
>
>          Time Spent: 20m
>  Remaining Estimate: 0h
>
> https://api.travis-ci.org/v3/job/564925128/log.txt
> {code}
> ------------------------------------------------------------
>  The program finished with the following exception:
> org.apache.flink.client.program.ProgramInvocationException: Job failed. 
> (JobID: 1b4f1807cc749628cfc1bdf04647527a)
>       at 
> org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:250)
>       at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:338)
>       at 
> org.apache.flink.streaming.api.environment.StreamContextEnvironment.execute(StreamContextEnvironment.java:60)
>       at 
> org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.execute(StreamExecutionEnvironment.java:1507)
>       at 
> org.apache.flink.deployment.HeavyDeploymentStressTestProgram.main(HeavyDeploymentStressTestProgram.java:70)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:576)
>       at 
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:438)
>       at 
> org.apache.flink.client.program.ClusterClient.run(ClusterClient.java:274)
>       at 
> org.apache.flink.client.cli.CliFrontend.executeProgram(CliFrontend.java:746)
>       at 
> org.apache.flink.client.cli.CliFrontend.runProgram(CliFrontend.java:273)
>       at org.apache.flink.client.cli.CliFrontend.run(CliFrontend.java:205)
>       at 
> org.apache.flink.client.cli.CliFrontend.parseParameters(CliFrontend.java:1010)
>       at 
> org.apache.flink.client.cli.CliFrontend.lambda$main$10(CliFrontend.java:1083)
>       at java.security.AccessController.doPrivileged(Native Method)
>       at javax.security.auth.Subject.doAs(Subject.java:422)
>       at 
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1836)
>       at 
> org.apache.flink.runtime.security.HadoopSecurityContext.runSecured(HadoopSecurityContext.java:41)
>       at org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1083)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Job 
> execution failed.
>       at 
> org.apache.flink.runtime.jobmaster.JobResult.toJobExecutionResult(JobResult.java:146)
>       at 
> org.apache.flink.client.program.rest.RestClusterClient.submitJob(RestClusterClient.java:247)
>       ... 21 more
> Caused by: java.util.concurrent.TimeoutException: Heartbeat of TaskManager 
> with id ea456d6a590eca7598c19c4d35e56db9 timed out.
>       at 
> org.apache.flink.runtime.jobmaster.JobMaster$TaskManagerHeartbeatListener.notifyHeartbeatTimeout(JobMaster.java:1149)
>       at 
> org.apache.flink.runtime.heartbeat.HeartbeatManagerImpl$HeartbeatMonitor.run(HeartbeatManagerImpl.java:318)
>       at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>       at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>       at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)
>       at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)
>       at 
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)
>       at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)
>       at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)
>       at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)
>       at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
>       at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)
>       at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)
>       at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
>       at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)
>       at akka.actor.Actor$class.aroundReceive(Actor.scala:517)
>       at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)
>       at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)
>       at akka.actor.ActorCell.invoke(ActorCell.scala:561)
>       at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)
>       at akka.dispatch.Mailbox.run(Mailbox.scala:225)
>       at akka.dispatch.Mailbox.exec(Mailbox.scala:235)
>       at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>       at 
> akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>       at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>       at 
> akka.dispatch.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to