The amount of resource assigned to JobManager looks fine. How much resource
(CPU and memory) did you allocate for TaskManager?

On Wed, Jul 19, 2017 at 3:07 AM, Stephan Ewen <se...@apache.org> wrote:

> Hi Greg!
>
> Akka disassociation means that the network connection between the
> JobManager and TaskManager broke.
>
> This can be cause by
>  - actual failures of JobManager / TaskManager (I assume is not the case
> here)
>  - A limit in the number of open file handles
>  - Network flakeyness
>  - I have seen cases where it looks like it was caused by network overload
> where shuffles basically starve/suppress the akka network connections
>
>
> Handling this in akka is not very nice - one does not get an easy way to
> actually deal with the root issues, but the actor systems sort of hides
> these things. This is yet one more reason why I am thinking to move away
> from akka - it may simply not be the best match for what we are doing.
>
>
> Greetings,
> Stephan
>
>
> On Fri, Jul 14, 2017 at 7:11 PM, Greg Hogan <c...@greghogan.com> wrote:
>
> > Hi all,
> >
> > I’m having some issues with Akka running on a modest cluster where
> > increasing the parallelism results in disassociation messages.
> >
> > I am running a batch job, Gelly’s TriangleListing (for simplicity) which
> > is join-based. I have not seen this issue running AdamicAdar which is
> > sort-based.
> >
> > I have increased both of the following timeouts and the job takes less
> > than 100 seconds.
> > akka.ask.timeout: 1000 s
> > akka.lookup.timeout: 100 s
> >
> > I have not changed taskmanager.exit-on-fatal-akka-error from the default
> > value of false but the JobManager is dropping all TaskManager
> connections.
> >
> > I can run the TriangleListing job with the same 127 TaskManagers with a
> > smaller parallelism. Dropping from 2286 to around 1000 is often
> successful.
> >
> > CPU and memory should not be a bottleneck for the JobManager (18 cores
> and
> > 18 GB).
> >
> > I would be grateful for solutions, suggestions, or pointers to debugging
> > this issue.
> >
> > Thanks,
> > Greg
> >
> >
> > 2017-07-14 16:50:08,119 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (30/2286) (
> > 5a2e8f0a00530bd2216d7d3ee10688f7) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:08,312 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (26/2286) (
> > c6a91db2d6b6797768596d9f746d316f) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:09,831 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (131/2286) (
> > 2c77b1e4b90b951d3be1e09bf4cf41d2) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:10,057 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (133/2286) (
> > d0c4c4eda4f0c44fe594a1b94eb66c93) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:11,861 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (70/2286) (
> > 69ce8d91fbbad943c277ee92d3c38aaa) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:15,029 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (38/2286) (
> > a72c2dee009342bc4d90ec98427fa717) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:16,583 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (27/2286) (
> > e79ec6229d4afdc6669c1c221a19ad8c) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:19,498 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - GroupReduce (Generate triplets) (44/2286) (
> > 53e35ddbd0e02d256620e5310276bea6) switched from RUNNING to FINISHED.
> > 2017-07-14 16:50:21,021 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-28-115:40713] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,097 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-21-141:45899] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,129 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-27-236:37471] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,132 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-18-79:45765] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,140 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-29-112:41017] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,142 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-25-70:39625] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,159 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-28-105:39127] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,170 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-28-117:38923] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,181 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-20-172:40007] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,190 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-22-220:44391] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,215 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-25-130:34963] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,219 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-18-125:38015] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,227 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-17-222:42125] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,228 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-25-92:38905] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,229 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-21-198:35983] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,233 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-18-217:41155] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,238 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-30-198:35671] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,238 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-27-213:44085] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,245 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-17-64:42935] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,268 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-28-99:37419] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,287 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-31-97:36659] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,288 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-24-180:36329] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,290 WARN  akka.remote.ReliableDeliverySupervisor
> >                   - Association with remote system
> > [akka.tcp://flink@ip-10-0-16-158:44247] has failed, address is now gated
> > for [5000] ms. Reason: [Disassociated]
> > 2017-07-14 16:50:21,293 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - Join (Triangle listing) (990/2286) (
> a2df3b72293f636bb4434f4e2fe342d7)
> > switched from RUNNING to FAILED.
> > org.apache.flink.runtime.io.network.netty.exception.
> RemoteTransportException:
> > Error at remote task manager 'ip-10-0-25-70/10.0.25.70:43941'.
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestClientHandler.decodeMsg(PartitionRequestClientHandler.
> > java:253)
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestClientHandler.channelRead(PartitionRequestClientHandler.
> > java:160)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeChannelRead(AbstractChannelHandlerContext.java:339)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> fireChannelRead(
> > AbstractChannelHandlerContext.java:324)
> >         at io.netty.handler.codec.MessageToMessageDecoder.channelRead(
> > MessageToMessageDecoder.java:103)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeChannelRead(AbstractChannelHandlerContext.java:339)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> fireChannelRead(
> > AbstractChannelHandlerContext.java:324)
> >         at io.netty.handler.codec.ByteToMessageDecoder.channelRead(
> > ByteToMessageDecoder.java:242)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeChannelRead(AbstractChannelHandlerContext.java:339)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> fireChannelRead(
> > AbstractChannelHandlerContext.java:324)
> >         at io.netty.channel.DefaultChannelPipeline.fireChannelRead(
> > DefaultChannelPipeline.java:847)
> >         at io.netty.channel.nio.AbstractNioByteChannel$
> NioByteUnsafe.read(
> > AbstractNioByteChannel.java:131)
> >         at io.netty.channel.nio.NioEventLoop.processSelectedKey(
> > NioEventLoop.java:511)
> >         at io.netty.channel.nio.NioEventLoop.
> processSelectedKeysOptimized(
> > NioEventLoop.java:468)
> >         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(
> > NioEventLoop.java:382)
> >         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
> >         at io.netty.util.concurrent.SingleThreadEventExecutor$2.
> > run(SingleThreadEventExecutor.java:111)
> >         at java.lang.Thread.run(Thread.java:745)
> > Caused by: org.apache.flink.runtime.io.network.partition.
> ProducerFailedException:
> > java.lang.Exception: TaskManager akka://flink/user/taskmanager
> disconnects
> > from JobManager akka.tcp://flink@10.0.25.151:6123/user/jobmanager:
> > JobManager is no longer reachable
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestQueue.writeAndFlushNextMessageIfPoss
> > ible(PartitionRequestQueue.java:170)
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestQueue.userEventTriggered(PartitionRequestQueue.java:111)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.ChannelInboundHandlerAdapter.
> > userEventTriggered(ChannelInboundHandlerAdapter.java:108)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.ChannelInboundHandlerAdapter.
> > userEventTriggered(ChannelInboundHandlerAdapter.java:108)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.ChannelInboundHandlerAdapter.
> > userEventTriggered(ChannelInboundHandlerAdapter.java:108)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.DefaultChannelPipeline.
> fireUserEventTriggered(
> > DefaultChannelPipeline.java:841)
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestQueue$1.run(PartitionRequestQueue.java:84)
> >         at io.netty.util.concurrent.SingleThreadEventExecutor.
> runAllTasks(
> > SingleThreadEventExecutor.java:357)
> >         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> >         ... 2 more
> > Caused by: java.lang.Exception: TaskManager akka://flink/user/taskmanager
> > disconnects from JobManager akka.tcp://flink@10.0.25.151:
> > 6123/user/jobmanager: JobManager is no longer reachable
> >         at org.apache.flink.runtime.taskmanager.TaskManager.
> > handleJobManagerDisconnect(TaskManager.scala:1096)
> >         at org.apache.flink.runtime.taskmanager.TaskManager$$
> > anonfun$handleMessage$1.applyOrElse(TaskManager.scala:312)
> >         at scala.runtime.AbstractPartialFunction.apply(
> > AbstractPartialFunction.scala:36)
> >         at org.apache.flink.runtime.LeaderSessionMessageFilter$$
> > anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:49)
> >         at scala.runtime.AbstractPartialFunction.apply(
> > AbstractPartialFunction.scala:36)
> >         at org.apache.flink.runtime.LogMessages$$anon$1.apply(
> > LogMessages.scala:33)
> >         at org.apache.flink.runtime.LogMessages$$anon$1.apply(
> > LogMessages.scala:28)
> >         at scala.PartialFunction$class.applyOrElse(PartialFunction.
> > scala:123)
> >         at org.apache.flink.runtime.LogMessages$$anon$1.
> > applyOrElse(LogMessages.scala:28)
> >         at akka.actor.Actor$class.aroundReceive(Actor.scala:467)
> >         at org.apache.flink.runtime.taskmanager.TaskManager.
> > aroundReceive(TaskManager.scala:121)
> >         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> >         at akka.actor.dungeon.DeathWatch$class.receivedTerminated(
> > DeathWatch.scala:44)
> >         at akka.actor.ActorCell.receivedTerminated(ActorCell.scala:369)
> >         at akka.actor.ActorCell.autoReceiveMessage(ActorCell.scala:501)
> >         at akka.actor.ActorCell.invoke(ActorCell.scala:486)
> >         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
> >         at akka.dispatch.Mailbox.run(Mailbox.scala:220)
> >         at akka.dispatch.ForkJoinExecutorConfigurator$
> > AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
> >         at scala.concurrent.forkjoin.ForkJoinTask.doExec(
> > ForkJoinTask.java:260)
> >         at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> > runTask(ForkJoinPool.java:1339)
> >         at scala.concurrent.forkjoin.ForkJoinPool.runWorker(
> > ForkJoinPool.java:1979)
> >         at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(
> > ForkJoinWorkerThread.java:107)
> > 2017-07-14 16:50:21,293 INFO  org.apache.flink.runtime.
> executiongraph.ExecutionGraph
> >       - Join (Triangle listing) (11/2286) (
> cc78687ccb6581e191c7f9ca24f7ab38)
> > switched from RUNNING to FAILED.
> > org.apache.flink.runtime.io.network.netty.exception.
> RemoteTransportException:
> > Error at remote task manager 'ip-10-0-25-70/10.0.25.70:43941'.
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestClientHandler.decodeMsg(PartitionRequestClientHandler.
> > java:253)
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestClientHandler.channelRead(PartitionRequestClientHandler.
> > java:160)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeChannelRead(AbstractChannelHandlerContext.java:339)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> fireChannelRead(
> > AbstractChannelHandlerContext.java:324)
> >         at io.netty.handler.codec.MessageToMessageDecoder.channelRead(
> > MessageToMessageDecoder.java:103)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeChannelRead(AbstractChannelHandlerContext.java:339)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> fireChannelRead(
> > AbstractChannelHandlerContext.java:324)
> >         at io.netty.handler.codec.ByteToMessageDecoder.channelRead(
> > ByteToMessageDecoder.java:242)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeChannelRead(AbstractChannelHandlerContext.java:339)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> fireChannelRead(
> > AbstractChannelHandlerContext.java:324)
> >         at io.netty.channel.DefaultChannelPipeline.fireChannelRead(
> > DefaultChannelPipeline.java:847)
> >         at io.netty.channel.nio.AbstractNioByteChannel$
> NioByteUnsafe.read(
> > AbstractNioByteChannel.java:131)
> >         at io.netty.channel.nio.NioEventLoop.processSelectedKey(
> > NioEventLoop.java:511)
> >         at io.netty.channel.nio.NioEventLoop.
> processSelectedKeysOptimized(
> > NioEventLoop.java:468)
> >         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(
> > NioEventLoop.java:382)
> >         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
> >         at io.netty.util.concurrent.SingleThreadEventExecutor$2.
> > run(SingleThreadEventExecutor.java:111)
> >         at java.lang.Thread.run(Thread.java:745)
> > Caused by: org.apache.flink.runtime.io.network.partition.
> ProducerFailedException:
> > java.lang.Exception: TaskManager akka://flink/user/taskmanager
> disconnects
> > from JobManager akka.tcp://flink@10.0.25.151:6123/user/jobmanager:
> > JobManager is no longer reachable
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestQueue.writeAndFlushNextMessageIfPoss
> > ible(PartitionRequestQueue.java:170)
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestQueue.userEventTriggered(PartitionRequestQueue.java:111)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.ChannelInboundHandlerAdapter.
> > userEventTriggered(ChannelInboundHandlerAdapter.java:108)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.ChannelInboundHandlerAdapter.
> > userEventTriggered(ChannelInboundHandlerAdapter.java:108)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.ChannelInboundHandlerAdapter.
> > userEventTriggered(ChannelInboundHandlerAdapter.java:108)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > invokeUserEventTriggered(AbstractChannelHandlerContext.java:308)
> >         at io.netty.channel.AbstractChannelHandlerContext.
> > fireUserEventTriggered(AbstractChannelHandlerContext.java:294)
> >         at io.netty.channel.DefaultChannelPipeline.
> fireUserEventTriggered(
> > DefaultChannelPipeline.java:841)
> >         at org.apache.flink.runtime.io.network.netty.
> > PartitionRequestQueue$1.run(PartitionRequestQueue.java:84)
> >         at io.netty.util.concurrent.SingleThreadEventExecutor.
> runAllTasks(
> > SingleThreadEventExecutor.java:357)
> >         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> >         ... 2 more
> > Caused by: java.lang.Exception: TaskManager akka://flink/user/taskmanager
> > disconnects from JobManager akka.tcp://flink@10.0.25.151:
> > 6123/user/jobmanager: JobManager is no longer reachable
> >         at org.apache.flink.runtime.taskmanager.TaskManager.
> > handleJobManagerDisconnect(TaskManager.scala:1096)
> >         at org.apache.flink.runtime.taskmanager.TaskManager$$
> > anonfun$handleMessage$1.applyOrElse(TaskManager.scala:312)
> >         at scala.runtime.AbstractPartialFunction.apply(
> > AbstractPartialFunction.scala:36)
> >         at org.apache.flink.runtime.LeaderSessionMessageFilter$$
> > anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:49)
> >         at scala.runtime.AbstractPartialFunction.apply(
> > AbstractPartialFunction.scala:36)
> >         at org.apache.flink.runtime.LogMessages$$anon$1.apply(
> > LogMessages.scala:33)
> >         at org.apache.flink.runtime.LogMessages$$anon$1.apply(
> > LogMessages.scala:28)
> >         at scala.PartialFunction$class.applyOrElse(PartialFunction.
> > scala:123)
> >         at org.apache.flink.runtime.LogMessages$$anon$1.
> > applyOrElse(LogMessages.scala:28)
> >         at akka.actor.Actor$class.aroundReceive(Actor.scala:467)
> >         at org.apache.flink.runtime.taskmanager.TaskManager.
> > aroundReceive(TaskManager.scala:121)
> >         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> >         at akka.actor.dungeon.DeathWatch$class.receivedTerminated(
> > DeathWatch.scala:44)
> >         at akka.actor.ActorCell.receivedTerminated(ActorCell.scala:369)
> >         at akka.actor.ActorCell.autoReceiveMessage(ActorCell.scala:501)
> >         at akka.actor.ActorCell.invoke(ActorCell.scala:486)
> >         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:238)
> >         at akka.dispatch.Mailbox.run(Mailbox.scala:220)
> >         at akka.dispatch.ForkJoinExecutorConfigurator$
> > AkkaForkJoinTask.exec(AbstractDispatcher.scala:397)
> >         at scala.concurrent.forkjoin.ForkJoinTask.doExec(
> > ForkJoinTask.java:260)
> >         at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.
> > runTask(ForkJoinPool.java:1339)
> >         at scala.concurrent.forkjoin.ForkJoinPool.runWorker(
> > ForkJoinPool.java:1979)
> >         at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(
> > ForkJoinWorkerThread.java:107)
> >
> >
>

Reply via email to