You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Greg Hogan <co...@greghogan.com> on 2015/10/29 13:46:15 UTC

Diagnosing TaskManager disappearance

I am testing again on a 64 node cluster (the JobManager is running fine
having reduced some operator's parallelism and fixed the string conversion
performance).

I am seeing TaskManagers drop like flies every other job or so. I am not
seeing any output in the .out log files corresponding to the crashed
TaskManagers.

Below is the stack trace from a java.hprof heap dump.

How should I be debugging this?

Thanks,
Greg


Threads at the heap dump:

Unknown thread


"Memory Logger" daemon prio=1 tid=119 TIMED_WAITING
at java.lang.Thread.<init>(Thread.java:507)
at
org.apache.flink.runtime.taskmanager.MemoryLogger.<init>(MemoryLogger.java:67)
at
org.apache.flink.runtime.taskmanager.TaskManager$.runTaskManager(TaskManager.scala:1494)
at
org.apache.flink.runtime.taskmanager.TaskManager$.selectNetworkInterfaceAndRunTaskManager(TaskManager.scala:1330)


"Flink Netty Server (59693) Thread 0" daemon prio=5 tid=193 RUNNABLE
at java.lang.Thread.<init>(Thread.java:674)
at
java.util.concurrent.Executors$DefaultThreadFactory.newThread(Executors.java:613)
at
org.apache.flink.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder$1.newThread(ThreadFactoryBuilder.java:162)
at
io.netty.util.concurrent.SingleThreadEventExecutor.<init>(SingleThreadEventExecutor.java:106)


"flink-akka.remote.default-remote-dispatcher-6" daemon prio=5 tid=30
TIMED_WAITING
at java.lang.Thread.<init>(Thread.java:507)
at
scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
at
akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)


"flink-akka.actor.default-dispatcher-4" daemon prio=5 tid=28 WAITING
at java.lang.Thread.<init>(Thread.java:507)
at
scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
at
akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)


"flink-akka.remote.default-remote-dispatcher-5" daemon prio=5 tid=29 WAITING
at java.lang.Thread.<init>(Thread.java:507)
at
scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
at
akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)


"flink-akka.actor.default-dispatcher-2" daemon prio=5 tid=26 WAITING
at java.lang.Thread.<init>(Thread.java:507)
at
scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
at
akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)


"SIGTERM handler" daemon prio=9 tid=268 RUNNABLE
at java.lang.Thread.<init>(Thread.java:547)
at sun.misc.Signal.dispatch(Signal.java:216)


"HPROF gc_finish watcher" daemon prio=10 tid=5 RUNNABLE


"Reference Handler" daemon prio=10 tid=2 WAITING


"main" prio=5 tid=1 WAITING


"Signal Dispatcher" daemon prio=9 tid=4 RUNNABLE


"Finalizer" daemon prio=8 tid=3 WAITING


"flink-akka.actor.default-dispatcher-3" daemon prio=5 tid=27 TIMED_WAITING
at java.lang.Thread.<init>(Thread.java:507)
at
scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
at
akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)


"New I/O worker #1" daemon prio=5 tid=31 RUNNABLE
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at
java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
at
java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)


"flink-scheduler-1" daemon prio=5 tid=25 TIMED_WAITING
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at akka.actor.LightArrayRevolverScheduler.<init>(Scheduler.scala:337)
at
sun.reflect.NativeConstructorAccessorImpl.newInstance0(NativeConstructorAccessorImpl.java)


"New I/O worker #2" daemon prio=5 tid=32 RUNNABLE
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at
java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
at
java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)


"Hashed wheel timer #1" daemon prio=5 tid=33 TIMED_WAITING
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at org.jboss.netty.util.HashedWheelTimer.<init>(HashedWheelTimer.java:226)
  Local Variable: java.util.ArrayList#502
at org.jboss.netty.util.HashedWheelTimer.<init>(HashedWheelTimer.java:177)
  Local Variable: java.lang.String#15234


"New I/O boss #3" daemon prio=5 tid=34 RUNNABLE
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at
java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
at
java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)


"Timer-0" daemon prio=5 tid=267 TIMED_WAITING
at java.lang.Thread.<init>(Thread.java:444)
at java.util.TimerThread.<init>(Timer.java:499)
at java.util.Timer.<init>(Timer.java:101)
at java.util.Timer.<init>(Timer.java:146)


"New I/O worker #4" daemon prio=5 tid=35 RUNNABLE
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at
java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
at
java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)


"New I/O worker #5" daemon prio=5 tid=36 RUNNABLE
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at
java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
at
java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)


"New I/O server boss #6" daemon prio=5 tid=37 RUNNABLE
at java.lang.Thread.<init>(Thread.java:547)
at
akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
at
java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
at
java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)

Re: Diagnosing TaskManager disappearance

Posted by Stephan Ewen <se...@apache.org>.
Hi!

The Netty memory usually goes much lower than 2*network memory (that is
theoretical).

Netty needs memory at the size two buffers on the sender and receiver side,
per TCP connection.
Since Flink usually multiplexes many Channels (that need network buffers)
through the same TCP connection, the
amount of memory actually needed by Netty is much lower than the
theoretical value.

That being said, we have seen also another case where the Netty memory size
is not trivially small.

A way to alleviate that is to allocate Flink's network memory off heap and
have the buffers implement Netty's ByteBuf interface.
That way, the sender side Netty channels will not need any additional
memory at all (direct zero copy Flink Buffer to Network stack), which is
quite a win already.

For the receiver side, it is a tad bit trickier, but afaik the receiver
Netty needs only memory in the size of one buffer per TCP channel,
so even without fixing this, we would be down to 1/3 of the required Netty
memory.


I think @uce has some thoughts about this as well...

Greetings,
Stephan





On Sat, Dec 12, 2015 at 12:53 PM, Greg Hogan <co...@greghogan.com> wrote:

> The TaskManagers were nixed by the OOM killer.
>
>   [63896.699500] Out of memory: Kill process 12892 (java) score 910 or
> sacrifice child
>   [63896.702018] Killed process 12892 (java) total-vm:47398740kB,
> anon-rss:28487812kB, file-rss:8kB
>
> The cluster is comprised of AWS c4.8xlarge instances which have 60 GiB of
> memory across two NUMA nodes (~32.2 GB each). Pertinent TaskManager
> configuration:
>
>   taskmanager.memory.off-heap: true
>   taskmanager.memory.segment-size: 16384
>   taskmanager.heap.mb: 18000
>   taskmanager.network.numberOfBuffers: 414720
>
> This was allocating 18 GB plus up to 6.8 GB for network buffers. As Max
> noted in FLINK-2865, "I think the maximum number of network memory can
> never exceed 2 * (network memory). In this case all network buffers would
> be inside the Netty buffer pool." Doubling the 6.8 GB exceeds the node
> memory.
>
> Would disabling off-heap memory cause the network buffers to be re-used by
> Netty and save half of the network buffer memory? I created FLINK-3164
> which would reduce the number of necessary network buffers.
>
> Greg Hogan
>
> On Fri, Oct 30, 2015 at 12:33 PM, Till Rohrmann <tr...@apache.org>
> wrote:
>
> > The logging of the TaskManager stops 3 seconds before the JobManager
> > detects that the connection to the TaskManager is failed. If the clocks
> are
> > remotely in sync and the TaskManager is still running, then we should
> also
> > see logging statements for the time after the connection has failed.
> > Therefore, I would also suspect that something happened to the
> TaskManager
> > JVM.
> >
> > Cheers,
> > Till
> >
> > On Fri, Oct 30, 2015 at 3:43 AM, Robert Metzger <rm...@apache.org>
> > wrote:
> >
> > > So is the TaskManager JVM still running after the JM detected that the
> TM
> > > has gone?
> > >
> > > If not, can you check the kernel log (dmesg) to see whether Linux OOM
> > > killer stopped the process? (if its a kill, the JVM might not be able
> to
> > > log anything anymore)
> > >
> > > On Thu, Oct 29, 2015 at 9:27 PM, Stephan Ewen <se...@apache.org>
> wrote:
> > >
> > > > Thanks for sharing the logs, Greg!
> > > >
> > > > Okay, so the TaskManager does not crash, but the Remote Failure
> > Detector
> > > of
> > > > Akka marks the connection between JobManager and TaskManager as
> broken.
> > > >
> > > > The TaskManager is not doing much GC, so it is not a long JVM freeze
> > that
> > > > causes hearbeats to time out...
> > > >
> > > > I am wondering at this point whether this is an issue in Akka,
> > > specifically
> > > > the remote death watch that we use to let the JobManager recognize
> > > > disconnected TaskManagers.
> > > >
> > > > One thing you could try is actually to comment out the line where the
> > > > JobManager starts the death watch for the TaskManager and see if they
> > can
> > > > still successfully exchange messages (tasks finished, find inputs,
> > > > schedule) and the program completes. That would indicate that the
> Akka
> > > > Death Watch is flawed and that we should probably do our own
> heartbeats
> > > > instead.
> > > >
> > > > Greetings,
> > > > Stephan
> > > >
> > > >
> > > > On Thu, Oct 29, 2015 at 11:44 AM, Aljoscha Krettek <
> > aljoscha@apache.org>
> > > > wrote:
> > > >
> > > > > Could it be a problem that there are two TaskManagers running per
> > > > machine?
> > > > >
> > > > > > On 29 Oct 2015, at 19:04, Greg Hogan <co...@greghogan.com> wrote:
> > > > > >
> > > > > > I have memory logging enabled. Tail of TaskManager log on
> > > 10.0.88.140:
> > > > > >
> > > > > > 17:35:26,415 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:27,415 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 576/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:27,415 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:27,415 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:28,012 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (938/2322)
> > > > > > 17:35:28,015 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (938/2322)
> > > > > > 17:35:28,016 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (938/2322) [DEPLOYING]
> > > > > > 17:35:28,065 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322)
> > > > switched
> > > > > to
> > > > > > RUNNING
> > > > > > 17:35:28,100 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (2304/2322)
> > > > > > 17:35:28,116 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (2304/2322)
> > > > > > 17:35:28,116 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (2304/2322) [DEPLOYING]
> > > > > > 17:35:28,132 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,255 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (939/2322)
> > > > > > 17:35:28,263 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (939/2322)
> > > > > > 17:35:28,263 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (939/2322) [DEPLOYING]
> > > > > > 17:35:28,304 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (2062/2322)
> > > > > > 17:35:28,311 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (2062/2322)
> > > > > > 17:35:28,311 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (2062/2322) [DEPLOYING]
> > > > > > 17:35:28,323 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322)
> > > > switched
> > > > > to
> > > > > > RUNNING
> > > > > > 17:35:28,386 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,396 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (1775/2322)
> > > > > > 17:35:28,401 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (1775/2322)
> > > > > > 17:35:28,402 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (1775/2322) [DEPLOYING]
> > > > > > 17:35:28,416 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:28,416 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:28,416 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:28,419 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,475 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (2158/2322)
> > > > > > 17:35:28,475 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (2158/2322)
> > > > > > 17:35:28,476 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (2158/2322) [DEPLOYING]
> > > > > > 17:35:28,509 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (1463/2322)
> > > > > > 17:35:28,860 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (1463/2322)
> > > > > > 17:35:28,861 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (1463/2322) [DEPLOYING]
> > > > > > 17:35:28,862 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,878 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,892 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (1154/2322)
> > > > > > 17:35:28,893 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (1154/2322)
> > > > > > 17:35:28,893 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (1154/2322) [DEPLOYING]
> > > > > > 17:35:28,914 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,916 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (1429/2322)
> > > > > > 17:35:28,917 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (1429/2322)
> > > > > > 17:35:28,917 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (1429/2322) [DEPLOYING]
> > > > > > 17:35:28,942 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (1078/2322)
> > > > > > 17:35:28,942 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (1078/2322)
> > > > > > 17:35:28,942 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (1078/2322) [DEPLOYING]
> > > > > > 17:35:28,943 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,955 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:28,959 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (524/2322)
> > > > > > 17:35:28,995 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (524/2322)
> > > > > > 17:35:28,995 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (524/2322) [DEPLOYING]
> > > > > > 17:35:29,000 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (2021/2322)
> > > > > > 17:35:29,000 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (2021/2322)
> > > > > > 17:35:29,000 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (2021/2322) [DEPLOYING]
> > > > > > 17:35:29,012 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322)
> > > > switched
> > > > > to
> > > > > > RUNNING
> > > > > > 17:35:29,039 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (2022/2322)
> > > > > > 17:35:29,039 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:29,043 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (2022/2322)
> > > > > > 17:35:29,043 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (2022/2322) [DEPLOYING]
> > > > > > 17:35:29,076 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (1464/2322)
> > > > > > 17:35:29,081 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (1464/2322)
> > > > > > 17:35:29,081 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (1464/2322) [DEPLOYING]
> > > > > > 17:35:29,095 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:29,108 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (1095/2322)
> > > > > > 17:35:29,110 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:29,112 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (1095/2322)
> > > > > > 17:35:29,112 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (1095/2322) [DEPLOYING]
> > > > > > 17:35:29,140 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (2306/2322)
> > > > > > 17:35:29,142 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (2306/2322)
> > > > > > 17:35:29,142 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (2306/2322) [DEPLOYING]
> > > > > > 17:35:29,147 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:29,152 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (974/2322)
> > > > > > 17:35:29,153 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:29,155 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (974/2322)
> > > > > > 17:35:29,155 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (974/2322) [DEPLOYING]
> > > > > > 17:35:29,166 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Received
> > > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > > (2305/2322)
> > > > > > 17:35:29,167 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Loading
> > > > > JAR
> > > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > > (2305/2322)
> > > > > > 17:35:29,167 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > > Registering
> > > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > > (checksum())
> > > > > > (2305/2322) [DEPLOYING]
> > > > > > 17:35:29,176 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322)
> > > > switched
> > > > > to
> > > > > > RUNNING
> > > > > > 17:35:29,205 INFO
> > > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > CHAIN
> > > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322)
> > > > switched
> > > > > > to RUNNING
> > > > > > 17:35:29,417 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 590/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:29,417 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:29,417 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:30,418 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 614/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:30,418 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:30,418 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:31,418 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 634/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:31,418 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:31,419 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:32,419 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 638/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:32,419 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:32,419 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:33,487 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:33,494 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:33,522 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:34,523 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 662/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:34,523 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:34,523 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:35,523 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 670/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:35,524 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:35,524 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:36,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 717/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:36,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:36,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:37,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 737/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:37,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:37,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:38,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:38,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:38,525 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:39,526 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 817/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:39,526 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:39,526 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:40,526 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 832/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:40,526 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:40,526 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:41,527 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 840/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:41,527 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:41,527 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:42,527 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 847/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:42,527 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:42,527 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:43,599 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 450/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:43,599 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:43,599 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:44,599 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 508/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:44,599 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:44,599 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:45,600 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 517/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:45,600 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:45,600 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:46,600 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 528/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:46,600 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:46,600 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:47,663 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 541/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:47,664 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:47,664 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:48,791 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 554/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:48,791 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:48,791 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:49,794 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 562/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:49,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:49,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:50,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 569/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:50,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:50,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:51,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 582/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:51,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:51,795 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:52,796 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 593/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:52,796 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:52,796 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:53,796 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 600/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:53,796 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:53,796 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:54,797 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 604/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:54,797 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:54,797 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:55,797 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 610/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:55,797 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:55,797 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:56,797 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 615/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:56,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:56,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:57,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 624/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:57,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:57,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:58,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 636/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:58,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:58,798 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:35:59,799 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 641/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:59,799 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:35:59,799 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:36:00,799 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:00,799 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:00,799 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:36:01,821 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 655/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:01,936 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:01,936 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:36:02,937 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 665/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:02,937 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:02,937 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > > 17:36:03,944 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Memory
> > > > > > usage stats: [HEAP: 666/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:03,950 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > > Off-heap
> > > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > > [Metaspace:
> > > > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space:
> > 4/4/1024
> > > MB
> > > > > > (used/committed/max)]
> > > > > > 17:36:03,951 INFO
> > > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Garbage
> > > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5],
> [PS
> > > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > >
> > > > > > On Thu, Oct 29, 2015 at 1:55 PM, Till Rohrmann <
> > trohrmann@apache.org
> > > >
> > > > > wrote:
> > > > > >
> > > > > >> What does the log of the failed TaskManager 10.0.88.140 say?
> > > > > >>
> > > > > >> On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <code@greghogan.com
> >
> > > > wrote:
> > > > > >>
> > > > > >>> I removed the use of numactl but left in starting two
> > TaskManagers
> > > > and
> > > > > am
> > > > > >>> still seeing TaskManagers crash.
> > > > > >>> From the JobManager log:
> > > > > >>>
> > > > > >>> 17:36:06,412 WARN
> > > > > >>> akka.remote.ReliableDeliverySupervisor                        -
> > > > > >> Association
> > > > > >>> with remote system [akka.tcp://flink@10.0.88.140:45742] has
> > > failed,
> > > > > >>> address
> > > > > >>> is now gated for [5000] ms. Reason is: [Disassociated].
> > > > > >>> 17:36:06,567 INFO
> > > > > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> > > CHAIN
> > > > > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
> > > > > >>> (cac9927a8568c2ad79439262a91478af) switched from RUNNING to
> > FAILED
> > > > > >>> 17:36:06,572 INFO
> > > > > >>> org.apache.flink.runtime.jobmanager.JobManager                -
> > > > Status
> > > > > of
> > > > > >>> job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct
> > 29
> > > > > >> 17:34:48
> > > > > >>> UTC 2015) changed to FAILING.
> > > > > >>> java.lang.Exception: The data preparation for task 'CHAIN
> > > GroupReduce
> > > > > >>> (Compute scores) -> FlatMap (checksum())' , caused an error:
> > Error
> > > > > >>> obtaining the sorted input: Thread 'SortMerger Reading Thread'
> > > > > terminated
> > > > > >>> due to an exception: Connection unexpectedly closed by remote
> > task
> > > > > >> manager
> > > > > >>> 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that
> the
> > > > > remote
> > > > > >>> task manager was lost.
> > > > > >>>        at
> > > > > >>>
> > > org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
> > > > > >>>        at
> > > > > >>>
> > > >
> org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
> > > > > >>>        at
> > > > org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> > > > > >>>        at java.lang.Thread.run(Thread.java:745)
> > > > > >>> Caused by: java.lang.RuntimeException: Error obtaining the
> sorted
> > > > > input:
> > > > > >>> Thread 'SortMerger Reading Thread' terminated due to an
> > exception:
> > > > > >>> Connection unexpectedly closed by remote task manager
> > > > 'ip-10-0-88-140/
> > > > > >>> 10.0.88.140:58558'. This might indicate that the remote task
> > > manager
> > > > > was
> > > > > >>> lost.
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
> > > > > >>>        at
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
> > > > > >>>        at
> > > > > >>>
> > > org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
> > > > > >>>        ... 3 more
> > > > > >>> Caused by: java.io.IOException: Thread 'SortMerger Reading
> > Thread'
> > > > > >>> terminated due to an exception: Connection unexpectedly closed
> by
> > > > > remote
> > > > > >>> task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might
> > > indicate
> > > > > >> that
> > > > > >>> the remote task manager was lost.
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
> > > > > >>> Caused by:
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
> > > > > >>> Connection unexpectedly closed by remote task manager
> > > > 'ip-10-0-88-140/
> > > > > >>> 10.0.88.140:58558'. This might indicate that the remote task
> > > manager
> > > > > was
> > > > > >>> lost.
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
> > > > > >>>        at
> > > > io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> > > > > >>>        at
> > > > > >>>
> > > > > >>>
> > > > > >>
> > > > >
> > > >
> > >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
> > > > > >>>        at java.lang.Thread.run(Thread.java:745)
> > > > > >>> 17:36:06,587 INFO
> > > > > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> > > CHAIN
> > > > > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
> > > > > >>> (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to
> > FAILED
> > > > > >>>
> > > > > >>>
> > > > > >>> On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <
> sewen@apache.org>
> > > > > wrote:
> > > > > >>>
> > > > > >>>> Hi Greg!
> > > > > >>>>
> > > > > >>>> Interesting... When you say the TaskManagers are dropping, are
> > the
> > > > > >>>> TaskManager processes crashing, or are they loosing connection
> > to
> > > > the
> > > > > >>>> JobManager?
> > > > > >>>>
> > > > > >>>> Greetings,
> > > > > >>>> Stephan
> > > > > >>>>
> > > > > >>>>
> > > > > >>>> On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <
> code@greghogan.com
> > >
> > > > > >> wrote:
> > > > > >>>>
> > > > > >>>>> I recently discovered that AWS uses NUMA for its largest
> nodes.
> > > An
> > > > > >>>> example
> > > > > >>>>> c4.8xlarge:
> > > > > >>>>>
> > > > > >>>>> $ numactl --hardware
> > > > > >>>>> available: 2 nodes (0-1)
> > > > > >>>>> node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> > > > > >>>>> node 0 size: 29813 MB
> > > > > >>>>> node 0 free: 24537 MB
> > > > > >>>>> node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33
> 34
> > 35
> > > > > >>>>> node 1 size: 30574 MB
> > > > > >>>>> node 1 free: 22757 MB
> > > > > >>>>> node distances:
> > > > > >>>>> node   0   1
> > > > > >>>>>  0:  10  20
> > > > > >>>>>  1:  20  10
> > > > > >>>>>
> > > > > >>>>> I discovered yesterday that Flink performed ~20-30% faster on
> > > large
> > > > > >>>>> datasets by running two NUMA-constrained TaskManagers per
> node.
> > > The
> > > > > >>>>> JobManager node ran a single TaskManager. Resources were
> > divided
> > > in
> > > > > >>> half
> > > > > >>>>> relative to running a single TaskManager.
> > > > > >>>>>
> > > > > >>>>> The changes from the tail of /bin/taskmanager.sh:
> > > > > >>>>>
> > > > > >>>>> -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager
> > > > > >> "${args[@]}"
> > > > > >>>>> +numactl --membind=0 --cpunodebind=0
> > > > > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > > > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > > > > >>>>> +numactl --membind=1 --cpunodebind=1
> > > > > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > > > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > > > > >>>>>
> > > > > >>>>> After reverting this change the system is again stable. I had
> > not
> > > > > >>>>> experienced issues using numactl when running 16 nodes.
> > > > > >>>>>
> > > > > >>>>> Greg
> > > > > >>>>>
> > > > > >>>>
> > > > > >>>
> > > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>

Re: Diagnosing TaskManager disappearance

Posted by Greg Hogan <co...@greghogan.com>.
The TaskManagers were nixed by the OOM killer.

  [63896.699500] Out of memory: Kill process 12892 (java) score 910 or
sacrifice child
  [63896.702018] Killed process 12892 (java) total-vm:47398740kB,
anon-rss:28487812kB, file-rss:8kB

The cluster is comprised of AWS c4.8xlarge instances which have 60 GiB of
memory across two NUMA nodes (~32.2 GB each). Pertinent TaskManager
configuration:

  taskmanager.memory.off-heap: true
  taskmanager.memory.segment-size: 16384
  taskmanager.heap.mb: 18000
  taskmanager.network.numberOfBuffers: 414720

This was allocating 18 GB plus up to 6.8 GB for network buffers. As Max
noted in FLINK-2865, "I think the maximum number of network memory can
never exceed 2 * (network memory). In this case all network buffers would
be inside the Netty buffer pool." Doubling the 6.8 GB exceeds the node
memory.

Would disabling off-heap memory cause the network buffers to be re-used by
Netty and save half of the network buffer memory? I created FLINK-3164
which would reduce the number of necessary network buffers.

Greg Hogan

On Fri, Oct 30, 2015 at 12:33 PM, Till Rohrmann <tr...@apache.org>
wrote:

> The logging of the TaskManager stops 3 seconds before the JobManager
> detects that the connection to the TaskManager is failed. If the clocks are
> remotely in sync and the TaskManager is still running, then we should also
> see logging statements for the time after the connection has failed.
> Therefore, I would also suspect that something happened to the TaskManager
> JVM.
>
> Cheers,
> Till
>
> On Fri, Oct 30, 2015 at 3:43 AM, Robert Metzger <rm...@apache.org>
> wrote:
>
> > So is the TaskManager JVM still running after the JM detected that the TM
> > has gone?
> >
> > If not, can you check the kernel log (dmesg) to see whether Linux OOM
> > killer stopped the process? (if its a kill, the JVM might not be able to
> > log anything anymore)
> >
> > On Thu, Oct 29, 2015 at 9:27 PM, Stephan Ewen <se...@apache.org> wrote:
> >
> > > Thanks for sharing the logs, Greg!
> > >
> > > Okay, so the TaskManager does not crash, but the Remote Failure
> Detector
> > of
> > > Akka marks the connection between JobManager and TaskManager as broken.
> > >
> > > The TaskManager is not doing much GC, so it is not a long JVM freeze
> that
> > > causes hearbeats to time out...
> > >
> > > I am wondering at this point whether this is an issue in Akka,
> > specifically
> > > the remote death watch that we use to let the JobManager recognize
> > > disconnected TaskManagers.
> > >
> > > One thing you could try is actually to comment out the line where the
> > > JobManager starts the death watch for the TaskManager and see if they
> can
> > > still successfully exchange messages (tasks finished, find inputs,
> > > schedule) and the program completes. That would indicate that the Akka
> > > Death Watch is flawed and that we should probably do our own heartbeats
> > > instead.
> > >
> > > Greetings,
> > > Stephan
> > >
> > >
> > > On Thu, Oct 29, 2015 at 11:44 AM, Aljoscha Krettek <
> aljoscha@apache.org>
> > > wrote:
> > >
> > > > Could it be a problem that there are two TaskManagers running per
> > > machine?
> > > >
> > > > > On 29 Oct 2015, at 19:04, Greg Hogan <co...@greghogan.com> wrote:
> > > > >
> > > > > I have memory logging enabled. Tail of TaskManager log on
> > 10.0.88.140:
> > > > >
> > > > > 17:35:26,415 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:27,415 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 576/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:27,415 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:27,415 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:28,012 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (938/2322)
> > > > > 17:35:28,015 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (938/2322)
> > > > > 17:35:28,016 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (938/2322) [DEPLOYING]
> > > > > 17:35:28,065 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322)
> > > switched
> > > > to
> > > > > RUNNING
> > > > > 17:35:28,100 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (2304/2322)
> > > > > 17:35:28,116 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (2304/2322)
> > > > > 17:35:28,116 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (2304/2322) [DEPLOYING]
> > > > > 17:35:28,132 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,255 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (939/2322)
> > > > > 17:35:28,263 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (939/2322)
> > > > > 17:35:28,263 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (939/2322) [DEPLOYING]
> > > > > 17:35:28,304 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (2062/2322)
> > > > > 17:35:28,311 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (2062/2322)
> > > > > 17:35:28,311 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (2062/2322) [DEPLOYING]
> > > > > 17:35:28,323 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322)
> > > switched
> > > > to
> > > > > RUNNING
> > > > > 17:35:28,386 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,396 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (1775/2322)
> > > > > 17:35:28,401 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (1775/2322)
> > > > > 17:35:28,402 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (1775/2322) [DEPLOYING]
> > > > > 17:35:28,416 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:28,416 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:28,416 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:28,419 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,475 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (2158/2322)
> > > > > 17:35:28,475 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (2158/2322)
> > > > > 17:35:28,476 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (2158/2322) [DEPLOYING]
> > > > > 17:35:28,509 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (1463/2322)
> > > > > 17:35:28,860 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (1463/2322)
> > > > > 17:35:28,861 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (1463/2322) [DEPLOYING]
> > > > > 17:35:28,862 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,878 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,892 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (1154/2322)
> > > > > 17:35:28,893 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (1154/2322)
> > > > > 17:35:28,893 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (1154/2322) [DEPLOYING]
> > > > > 17:35:28,914 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,916 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (1429/2322)
> > > > > 17:35:28,917 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (1429/2322)
> > > > > 17:35:28,917 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (1429/2322) [DEPLOYING]
> > > > > 17:35:28,942 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (1078/2322)
> > > > > 17:35:28,942 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (1078/2322)
> > > > > 17:35:28,942 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (1078/2322) [DEPLOYING]
> > > > > 17:35:28,943 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,955 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:28,959 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (524/2322)
> > > > > 17:35:28,995 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (524/2322)
> > > > > 17:35:28,995 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (524/2322) [DEPLOYING]
> > > > > 17:35:29,000 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (2021/2322)
> > > > > 17:35:29,000 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (2021/2322)
> > > > > 17:35:29,000 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (2021/2322) [DEPLOYING]
> > > > > 17:35:29,012 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322)
> > > switched
> > > > to
> > > > > RUNNING
> > > > > 17:35:29,039 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (2022/2322)
> > > > > 17:35:29,039 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:29,043 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (2022/2322)
> > > > > 17:35:29,043 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (2022/2322) [DEPLOYING]
> > > > > 17:35:29,076 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (1464/2322)
> > > > > 17:35:29,081 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (1464/2322)
> > > > > 17:35:29,081 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (1464/2322) [DEPLOYING]
> > > > > 17:35:29,095 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:29,108 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (1095/2322)
> > > > > 17:35:29,110 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:29,112 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (1095/2322)
> > > > > 17:35:29,112 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (1095/2322) [DEPLOYING]
> > > > > 17:35:29,140 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (2306/2322)
> > > > > 17:35:29,142 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (2306/2322)
> > > > > 17:35:29,142 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (2306/2322) [DEPLOYING]
> > > > > 17:35:29,147 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:29,152 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (974/2322)
> > > > > 17:35:29,153 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:29,155 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (974/2322)
> > > > > 17:35:29,155 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (974/2322) [DEPLOYING]
> > > > > 17:35:29,166 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Received
> > > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > > (2305/2322)
> > > > > 17:35:29,167 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > Loading
> > > > JAR
> > > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > > (2305/2322)
> > > > > 17:35:29,167 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > > Registering
> > > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > > (checksum())
> > > > > (2305/2322) [DEPLOYING]
> > > > > 17:35:29,176 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322)
> > > switched
> > > > to
> > > > > RUNNING
> > > > > 17:35:29,205 INFO
> > > > > org.apache.flink.runtime.taskmanager.Task                     -
> CHAIN
> > > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322)
> > > switched
> > > > > to RUNNING
> > > > > 17:35:29,417 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 590/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:29,417 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:29,417 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:30,418 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 614/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:30,418 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:30,418 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:31,418 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 634/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:31,418 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:31,419 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:32,419 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 638/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:32,419 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:32,419 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:33,487 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:33,494 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:33,522 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:34,523 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 662/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:34,523 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:34,523 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:35,523 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 670/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:35,524 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:35,524 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:36,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 717/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:36,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:36,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:37,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 737/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:37,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:37,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:38,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:38,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:38,525 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:39,526 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 817/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:39,526 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:39,526 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:40,526 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 832/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:40,526 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:40,526 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:41,527 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 840/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:41,527 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:41,527 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:42,527 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 847/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:42,527 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:42,527 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:43,599 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 450/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:43,599 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:43,599 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:44,599 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 508/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:44,599 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:44,599 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:45,600 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 517/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:45,600 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:45,600 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:46,600 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 528/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:46,600 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:46,600 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:47,663 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 541/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:47,664 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:47,664 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:48,791 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 554/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:48,791 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:48,791 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:49,794 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 562/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:49,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:49,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:50,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 569/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:50,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:50,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:51,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 582/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:51,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:51,795 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:52,796 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 593/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:52,796 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:52,796 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:53,796 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 600/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:53,796 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:53,796 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:54,797 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 604/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:54,797 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:54,797 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:55,797 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 610/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:55,797 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:55,797 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:56,797 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 615/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:56,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:56,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:57,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 624/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:57,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:57,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:58,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 636/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:58,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:58,798 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:35:59,799 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 641/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:35:59,799 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:35:59,799 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:36:00,799 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:36:00,799 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:36:00,799 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:36:01,821 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 655/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:36:01,936 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:36:01,936 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:36:02,937 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 665/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:36:02,937 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:36:02,937 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > > 17:36:03,944 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Memory
> > > > > usage stats: [HEAP: 666/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > > (used/committed/max)]
> > > > > 17:36:03,950 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > > Off-heap
> > > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > > [Metaspace:
> > > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space:
> 4/4/1024
> > MB
> > > > > (used/committed/max)]
> > > > > 17:36:03,951 INFO
> > > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Garbage
> > > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > >
> > > > > On Thu, Oct 29, 2015 at 1:55 PM, Till Rohrmann <
> trohrmann@apache.org
> > >
> > > > wrote:
> > > > >
> > > > >> What does the log of the failed TaskManager 10.0.88.140 say?
> > > > >>
> > > > >> On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <co...@greghogan.com>
> > > wrote:
> > > > >>
> > > > >>> I removed the use of numactl but left in starting two
> TaskManagers
> > > and
> > > > am
> > > > >>> still seeing TaskManagers crash.
> > > > >>> From the JobManager log:
> > > > >>>
> > > > >>> 17:36:06,412 WARN
> > > > >>> akka.remote.ReliableDeliverySupervisor                        -
> > > > >> Association
> > > > >>> with remote system [akka.tcp://flink@10.0.88.140:45742] has
> > failed,
> > > > >>> address
> > > > >>> is now gated for [5000] ms. Reason is: [Disassociated].
> > > > >>> 17:36:06,567 INFO
> > > > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> > CHAIN
> > > > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
> > > > >>> (cac9927a8568c2ad79439262a91478af) switched from RUNNING to
> FAILED
> > > > >>> 17:36:06,572 INFO
> > > > >>> org.apache.flink.runtime.jobmanager.JobManager                -
> > > Status
> > > > of
> > > > >>> job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct
> 29
> > > > >> 17:34:48
> > > > >>> UTC 2015) changed to FAILING.
> > > > >>> java.lang.Exception: The data preparation for task 'CHAIN
> > GroupReduce
> > > > >>> (Compute scores) -> FlatMap (checksum())' , caused an error:
> Error
> > > > >>> obtaining the sorted input: Thread 'SortMerger Reading Thread'
> > > > terminated
> > > > >>> due to an exception: Connection unexpectedly closed by remote
> task
> > > > >> manager
> > > > >>> 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the
> > > > remote
> > > > >>> task manager was lost.
> > > > >>>        at
> > > > >>>
> > org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
> > > > >>>        at
> > > > >>>
> > > org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
> > > > >>>        at
> > > org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> > > > >>>        at java.lang.Thread.run(Thread.java:745)
> > > > >>> Caused by: java.lang.RuntimeException: Error obtaining the sorted
> > > > input:
> > > > >>> Thread 'SortMerger Reading Thread' terminated due to an
> exception:
> > > > >>> Connection unexpectedly closed by remote task manager
> > > 'ip-10-0-88-140/
> > > > >>> 10.0.88.140:58558'. This might indicate that the remote task
> > manager
> > > > was
> > > > >>> lost.
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
> > > > >>>        at
> > > > >>>
> > > > >>
> > > >
> > >
> >
> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
> > > > >>>        at
> > > > >>>
> > org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
> > > > >>>        ... 3 more
> > > > >>> Caused by: java.io.IOException: Thread 'SortMerger Reading
> Thread'
> > > > >>> terminated due to an exception: Connection unexpectedly closed by
> > > > remote
> > > > >>> task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might
> > indicate
> > > > >> that
> > > > >>> the remote task manager was lost.
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
> > > > >>> Caused by:
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
> > > > >>> Connection unexpectedly closed by remote task manager
> > > 'ip-10-0-88-140/
> > > > >>> 10.0.88.140:58558'. This might indicate that the remote task
> > manager
> > > > was
> > > > >>> lost.
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
> > > > >>>        at
> > > io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> > > > >>>        at
> > > > >>>
> > > > >>>
> > > > >>
> > > >
> > >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
> > > > >>>        at java.lang.Thread.run(Thread.java:745)
> > > > >>> 17:36:06,587 INFO
> > > > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> > CHAIN
> > > > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
> > > > >>> (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to
> FAILED
> > > > >>>
> > > > >>>
> > > > >>> On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org>
> > > > wrote:
> > > > >>>
> > > > >>>> Hi Greg!
> > > > >>>>
> > > > >>>> Interesting... When you say the TaskManagers are dropping, are
> the
> > > > >>>> TaskManager processes crashing, or are they loosing connection
> to
> > > the
> > > > >>>> JobManager?
> > > > >>>>
> > > > >>>> Greetings,
> > > > >>>> Stephan
> > > > >>>>
> > > > >>>>
> > > > >>>> On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <code@greghogan.com
> >
> > > > >> wrote:
> > > > >>>>
> > > > >>>>> I recently discovered that AWS uses NUMA for its largest nodes.
> > An
> > > > >>>> example
> > > > >>>>> c4.8xlarge:
> > > > >>>>>
> > > > >>>>> $ numactl --hardware
> > > > >>>>> available: 2 nodes (0-1)
> > > > >>>>> node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> > > > >>>>> node 0 size: 29813 MB
> > > > >>>>> node 0 free: 24537 MB
> > > > >>>>> node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34
> 35
> > > > >>>>> node 1 size: 30574 MB
> > > > >>>>> node 1 free: 22757 MB
> > > > >>>>> node distances:
> > > > >>>>> node   0   1
> > > > >>>>>  0:  10  20
> > > > >>>>>  1:  20  10
> > > > >>>>>
> > > > >>>>> I discovered yesterday that Flink performed ~20-30% faster on
> > large
> > > > >>>>> datasets by running two NUMA-constrained TaskManagers per node.
> > The
> > > > >>>>> JobManager node ran a single TaskManager. Resources were
> divided
> > in
> > > > >>> half
> > > > >>>>> relative to running a single TaskManager.
> > > > >>>>>
> > > > >>>>> The changes from the tail of /bin/taskmanager.sh:
> > > > >>>>>
> > > > >>>>> -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager
> > > > >> "${args[@]}"
> > > > >>>>> +numactl --membind=0 --cpunodebind=0
> > > > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > > > >>>>> +numactl --membind=1 --cpunodebind=1
> > > > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > > > >>>>>
> > > > >>>>> After reverting this change the system is again stable. I had
> not
> > > > >>>>> experienced issues using numactl when running 16 nodes.
> > > > >>>>>
> > > > >>>>> Greg
> > > > >>>>>
> > > > >>>>
> > > > >>>
> > > > >>
> > > >
> > > >
> > >
> >
>

Re: Diagnosing TaskManager disappearance

Posted by Till Rohrmann <tr...@apache.org>.
The logging of the TaskManager stops 3 seconds before the JobManager
detects that the connection to the TaskManager is failed. If the clocks are
remotely in sync and the TaskManager is still running, then we should also
see logging statements for the time after the connection has failed.
Therefore, I would also suspect that something happened to the TaskManager
JVM.

Cheers,
Till

On Fri, Oct 30, 2015 at 3:43 AM, Robert Metzger <rm...@apache.org> wrote:

> So is the TaskManager JVM still running after the JM detected that the TM
> has gone?
>
> If not, can you check the kernel log (dmesg) to see whether Linux OOM
> killer stopped the process? (if its a kill, the JVM might not be able to
> log anything anymore)
>
> On Thu, Oct 29, 2015 at 9:27 PM, Stephan Ewen <se...@apache.org> wrote:
>
> > Thanks for sharing the logs, Greg!
> >
> > Okay, so the TaskManager does not crash, but the Remote Failure Detector
> of
> > Akka marks the connection between JobManager and TaskManager as broken.
> >
> > The TaskManager is not doing much GC, so it is not a long JVM freeze that
> > causes hearbeats to time out...
> >
> > I am wondering at this point whether this is an issue in Akka,
> specifically
> > the remote death watch that we use to let the JobManager recognize
> > disconnected TaskManagers.
> >
> > One thing you could try is actually to comment out the line where the
> > JobManager starts the death watch for the TaskManager and see if they can
> > still successfully exchange messages (tasks finished, find inputs,
> > schedule) and the program completes. That would indicate that the Akka
> > Death Watch is flawed and that we should probably do our own heartbeats
> > instead.
> >
> > Greetings,
> > Stephan
> >
> >
> > On Thu, Oct 29, 2015 at 11:44 AM, Aljoscha Krettek <al...@apache.org>
> > wrote:
> >
> > > Could it be a problem that there are two TaskManagers running per
> > machine?
> > >
> > > > On 29 Oct 2015, at 19:04, Greg Hogan <co...@greghogan.com> wrote:
> > > >
> > > > I have memory logging enabled. Tail of TaskManager log on
> 10.0.88.140:
> > > >
> > > > 17:35:26,415 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:27,415 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 576/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:27,415 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:27,415 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:28,012 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (938/2322)
> > > > 17:35:28,015 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (938/2322)
> > > > 17:35:28,016 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (938/2322) [DEPLOYING]
> > > > 17:35:28,065 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322)
> > switched
> > > to
> > > > RUNNING
> > > > 17:35:28,100 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (2304/2322)
> > > > 17:35:28,116 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (2304/2322)
> > > > 17:35:28,116 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (2304/2322) [DEPLOYING]
> > > > 17:35:28,132 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,255 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (939/2322)
> > > > 17:35:28,263 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (939/2322)
> > > > 17:35:28,263 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (939/2322) [DEPLOYING]
> > > > 17:35:28,304 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (2062/2322)
> > > > 17:35:28,311 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (2062/2322)
> > > > 17:35:28,311 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (2062/2322) [DEPLOYING]
> > > > 17:35:28,323 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322)
> > switched
> > > to
> > > > RUNNING
> > > > 17:35:28,386 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,396 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (1775/2322)
> > > > 17:35:28,401 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (1775/2322)
> > > > 17:35:28,402 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (1775/2322) [DEPLOYING]
> > > > 17:35:28,416 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:28,416 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:28,416 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:28,419 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,475 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (2158/2322)
> > > > 17:35:28,475 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (2158/2322)
> > > > 17:35:28,476 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (2158/2322) [DEPLOYING]
> > > > 17:35:28,509 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (1463/2322)
> > > > 17:35:28,860 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (1463/2322)
> > > > 17:35:28,861 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (1463/2322) [DEPLOYING]
> > > > 17:35:28,862 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,878 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,892 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (1154/2322)
> > > > 17:35:28,893 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (1154/2322)
> > > > 17:35:28,893 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (1154/2322) [DEPLOYING]
> > > > 17:35:28,914 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,916 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (1429/2322)
> > > > 17:35:28,917 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (1429/2322)
> > > > 17:35:28,917 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (1429/2322) [DEPLOYING]
> > > > 17:35:28,942 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (1078/2322)
> > > > 17:35:28,942 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (1078/2322)
> > > > 17:35:28,942 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (1078/2322) [DEPLOYING]
> > > > 17:35:28,943 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,955 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:28,959 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (524/2322)
> > > > 17:35:28,995 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (524/2322)
> > > > 17:35:28,995 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (524/2322) [DEPLOYING]
> > > > 17:35:29,000 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (2021/2322)
> > > > 17:35:29,000 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (2021/2322)
> > > > 17:35:29,000 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (2021/2322) [DEPLOYING]
> > > > 17:35:29,012 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322)
> > switched
> > > to
> > > > RUNNING
> > > > 17:35:29,039 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (2022/2322)
> > > > 17:35:29,039 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:29,043 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (2022/2322)
> > > > 17:35:29,043 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (2022/2322) [DEPLOYING]
> > > > 17:35:29,076 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (1464/2322)
> > > > 17:35:29,081 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (1464/2322)
> > > > 17:35:29,081 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (1464/2322) [DEPLOYING]
> > > > 17:35:29,095 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:29,108 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (1095/2322)
> > > > 17:35:29,110 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:29,112 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (1095/2322)
> > > > 17:35:29,112 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (1095/2322) [DEPLOYING]
> > > > 17:35:29,140 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (2306/2322)
> > > > 17:35:29,142 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (2306/2322)
> > > > 17:35:29,142 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (2306/2322) [DEPLOYING]
> > > > 17:35:29,147 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:29,152 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (974/2322)
> > > > 17:35:29,153 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:29,155 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (974/2322)
> > > > 17:35:29,155 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (974/2322) [DEPLOYING]
> > > > 17:35:29,166 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Received
> > > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > > (2305/2322)
> > > > 17:35:29,167 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> Loading
> > > JAR
> > > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > > (2305/2322)
> > > > 17:35:29,167 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     -
> > > Registering
> > > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > > (checksum())
> > > > (2305/2322) [DEPLOYING]
> > > > 17:35:29,176 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322)
> > switched
> > > to
> > > > RUNNING
> > > > 17:35:29,205 INFO
> > > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322)
> > switched
> > > > to RUNNING
> > > > 17:35:29,417 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 590/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:29,417 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:29,417 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:30,418 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 614/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:30,418 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:30,418 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:31,418 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 634/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:31,418 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:31,419 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:32,419 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 638/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:32,419 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:32,419 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:33,487 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:33,494 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:33,522 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:34,523 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 662/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:34,523 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:34,523 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:35,523 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 670/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:35,524 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:35,524 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:36,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 717/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:36,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:36,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:37,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 737/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:37,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:37,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:38,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:38,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:38,525 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:39,526 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 817/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:39,526 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:39,526 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:40,526 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 832/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:40,526 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:40,526 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:41,527 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 840/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:41,527 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:41,527 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:42,527 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 847/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:42,527 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:42,527 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:43,599 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 450/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:43,599 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:43,599 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:44,599 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 508/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:44,599 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:44,599 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:45,600 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 517/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:45,600 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:45,600 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:46,600 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 528/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:46,600 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:46,600 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:47,663 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 541/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:47,664 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:47,664 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:48,791 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 554/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:48,791 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:48,791 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:49,794 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 562/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:49,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:49,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:50,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 569/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:50,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:50,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:51,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 582/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:51,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:51,795 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:52,796 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 593/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:52,796 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:52,796 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:53,796 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 600/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:53,796 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:53,796 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:54,797 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 604/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:54,797 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:54,797 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:55,797 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 610/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:55,797 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:55,797 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:56,797 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 615/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:56,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:56,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:57,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 624/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:57,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:57,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:58,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 636/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:58,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:58,798 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:35:59,799 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 641/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:35:59,799 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:35:59,799 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:36:00,799 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > > (used/committed/max)]
> > > > 17:36:00,799 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:36:00,799 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:36:01,821 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 655/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > (used/committed/max)]
> > > > 17:36:01,936 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:36:01,936 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:36:02,937 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 665/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > (used/committed/max)]
> > > > 17:36:02,937 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:36:02,937 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > > 17:36:03,944 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Memory
> > > > usage stats: [HEAP: 666/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > > (used/committed/max)]
> > > > 17:36:03,950 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> > Off-heap
> > > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> > [Metaspace:
> > > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024
> MB
> > > > (used/committed/max)]
> > > > 17:36:03,951 INFO
> > > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Garbage
> > > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > >
> > > > On Thu, Oct 29, 2015 at 1:55 PM, Till Rohrmann <trohrmann@apache.org
> >
> > > wrote:
> > > >
> > > >> What does the log of the failed TaskManager 10.0.88.140 say?
> > > >>
> > > >> On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <co...@greghogan.com>
> > wrote:
> > > >>
> > > >>> I removed the use of numactl but left in starting two TaskManagers
> > and
> > > am
> > > >>> still seeing TaskManagers crash.
> > > >>> From the JobManager log:
> > > >>>
> > > >>> 17:36:06,412 WARN
> > > >>> akka.remote.ReliableDeliverySupervisor                        -
> > > >> Association
> > > >>> with remote system [akka.tcp://flink@10.0.88.140:45742] has
> failed,
> > > >>> address
> > > >>> is now gated for [5000] ms. Reason is: [Disassociated].
> > > >>> 17:36:06,567 INFO
> > > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> CHAIN
> > > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
> > > >>> (cac9927a8568c2ad79439262a91478af) switched from RUNNING to FAILED
> > > >>> 17:36:06,572 INFO
> > > >>> org.apache.flink.runtime.jobmanager.JobManager                -
> > Status
> > > of
> > > >>> job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct 29
> > > >> 17:34:48
> > > >>> UTC 2015) changed to FAILING.
> > > >>> java.lang.Exception: The data preparation for task 'CHAIN
> GroupReduce
> > > >>> (Compute scores) -> FlatMap (checksum())' , caused an error: Error
> > > >>> obtaining the sorted input: Thread 'SortMerger Reading Thread'
> > > terminated
> > > >>> due to an exception: Connection unexpectedly closed by remote task
> > > >> manager
> > > >>> 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the
> > > remote
> > > >>> task manager was lost.
> > > >>>        at
> > > >>>
> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
> > > >>>        at
> > > >>>
> > org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
> > > >>>        at
> > org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> > > >>>        at java.lang.Thread.run(Thread.java:745)
> > > >>> Caused by: java.lang.RuntimeException: Error obtaining the sorted
> > > input:
> > > >>> Thread 'SortMerger Reading Thread' terminated due to an exception:
> > > >>> Connection unexpectedly closed by remote task manager
> > 'ip-10-0-88-140/
> > > >>> 10.0.88.140:58558'. This might indicate that the remote task
> manager
> > > was
> > > >>> lost.
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
> > > >>>        at
> > > >>>
> > > >>
> > >
> >
> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
> > > >>>        at
> > > >>>
> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
> > > >>>        ... 3 more
> > > >>> Caused by: java.io.IOException: Thread 'SortMerger Reading Thread'
> > > >>> terminated due to an exception: Connection unexpectedly closed by
> > > remote
> > > >>> task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might
> indicate
> > > >> that
> > > >>> the remote task manager was lost.
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
> > > >>> Caused by:
> > > >>>
> > > >>>
> > > >>
> > >
> >
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
> > > >>> Connection unexpectedly closed by remote task manager
> > 'ip-10-0-88-140/
> > > >>> 10.0.88.140:58558'. This might indicate that the remote task
> manager
> > > was
> > > >>> lost.
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
> > > >>>        at
> > io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> > > >>>        at
> > > >>>
> > > >>>
> > > >>
> > >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
> > > >>>        at java.lang.Thread.run(Thread.java:745)
> > > >>> 17:36:06,587 INFO
> > > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        -
> CHAIN
> > > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
> > > >>> (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to FAILED
> > > >>>
> > > >>>
> > > >>> On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org>
> > > wrote:
> > > >>>
> > > >>>> Hi Greg!
> > > >>>>
> > > >>>> Interesting... When you say the TaskManagers are dropping, are the
> > > >>>> TaskManager processes crashing, or are they loosing connection to
> > the
> > > >>>> JobManager?
> > > >>>>
> > > >>>> Greetings,
> > > >>>> Stephan
> > > >>>>
> > > >>>>
> > > >>>> On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com>
> > > >> wrote:
> > > >>>>
> > > >>>>> I recently discovered that AWS uses NUMA for its largest nodes.
> An
> > > >>>> example
> > > >>>>> c4.8xlarge:
> > > >>>>>
> > > >>>>> $ numactl --hardware
> > > >>>>> available: 2 nodes (0-1)
> > > >>>>> node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> > > >>>>> node 0 size: 29813 MB
> > > >>>>> node 0 free: 24537 MB
> > > >>>>> node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
> > > >>>>> node 1 size: 30574 MB
> > > >>>>> node 1 free: 22757 MB
> > > >>>>> node distances:
> > > >>>>> node   0   1
> > > >>>>>  0:  10  20
> > > >>>>>  1:  20  10
> > > >>>>>
> > > >>>>> I discovered yesterday that Flink performed ~20-30% faster on
> large
> > > >>>>> datasets by running two NUMA-constrained TaskManagers per node.
> The
> > > >>>>> JobManager node ran a single TaskManager. Resources were divided
> in
> > > >>> half
> > > >>>>> relative to running a single TaskManager.
> > > >>>>>
> > > >>>>> The changes from the tail of /bin/taskmanager.sh:
> > > >>>>>
> > > >>>>> -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager
> > > >> "${args[@]}"
> > > >>>>> +numactl --membind=0 --cpunodebind=0
> > > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > > >>>>> +numactl --membind=1 --cpunodebind=1
> > > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > > >>>>>
> > > >>>>> After reverting this change the system is again stable. I had not
> > > >>>>> experienced issues using numactl when running 16 nodes.
> > > >>>>>
> > > >>>>> Greg
> > > >>>>>
> > > >>>>
> > > >>>
> > > >>
> > >
> > >
> >
>

Re: Diagnosing TaskManager disappearance

Posted by Robert Metzger <rm...@apache.org>.
So is the TaskManager JVM still running after the JM detected that the TM
has gone?

If not, can you check the kernel log (dmesg) to see whether Linux OOM
killer stopped the process? (if its a kill, the JVM might not be able to
log anything anymore)

On Thu, Oct 29, 2015 at 9:27 PM, Stephan Ewen <se...@apache.org> wrote:

> Thanks for sharing the logs, Greg!
>
> Okay, so the TaskManager does not crash, but the Remote Failure Detector of
> Akka marks the connection between JobManager and TaskManager as broken.
>
> The TaskManager is not doing much GC, so it is not a long JVM freeze that
> causes hearbeats to time out...
>
> I am wondering at this point whether this is an issue in Akka, specifically
> the remote death watch that we use to let the JobManager recognize
> disconnected TaskManagers.
>
> One thing you could try is actually to comment out the line where the
> JobManager starts the death watch for the TaskManager and see if they can
> still successfully exchange messages (tasks finished, find inputs,
> schedule) and the program completes. That would indicate that the Akka
> Death Watch is flawed and that we should probably do our own heartbeats
> instead.
>
> Greetings,
> Stephan
>
>
> On Thu, Oct 29, 2015 at 11:44 AM, Aljoscha Krettek <al...@apache.org>
> wrote:
>
> > Could it be a problem that there are two TaskManagers running per
> machine?
> >
> > > On 29 Oct 2015, at 19:04, Greg Hogan <co...@greghogan.com> wrote:
> > >
> > > I have memory logging enabled. Tail of TaskManager log on 10.0.88.140:
> > >
> > > 17:35:26,415 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:27,415 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 576/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > (used/committed/max)]
> > > 17:35:27,415 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:27,415 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:28,012 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (938/2322)
> > > 17:35:28,015 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (938/2322)
> > > 17:35:28,016 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (938/2322) [DEPLOYING]
> > > 17:35:28,065 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322)
> switched
> > to
> > > RUNNING
> > > 17:35:28,100 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2304/2322)
> > > 17:35:28,116 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (2304/2322)
> > > 17:35:28,116 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (2304/2322) [DEPLOYING]
> > > 17:35:28,132 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322)
> switched
> > > to RUNNING
> > > 17:35:28,255 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (939/2322)
> > > 17:35:28,263 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (939/2322)
> > > 17:35:28,263 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (939/2322) [DEPLOYING]
> > > 17:35:28,304 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2062/2322)
> > > 17:35:28,311 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (2062/2322)
> > > 17:35:28,311 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (2062/2322) [DEPLOYING]
> > > 17:35:28,323 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322)
> switched
> > to
> > > RUNNING
> > > 17:35:28,386 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322)
> switched
> > > to RUNNING
> > > 17:35:28,396 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1775/2322)
> > > 17:35:28,401 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (1775/2322)
> > > 17:35:28,402 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (1775/2322) [DEPLOYING]
> > > 17:35:28,416 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > > (used/committed/max)]
> > > 17:35:28,416 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:28,416 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:28,419 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322)
> switched
> > > to RUNNING
> > > 17:35:28,475 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2158/2322)
> > > 17:35:28,475 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (2158/2322)
> > > 17:35:28,476 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (2158/2322) [DEPLOYING]
> > > 17:35:28,509 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1463/2322)
> > > 17:35:28,860 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (1463/2322)
> > > 17:35:28,861 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (1463/2322) [DEPLOYING]
> > > 17:35:28,862 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322)
> switched
> > > to RUNNING
> > > 17:35:28,878 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322)
> switched
> > > to RUNNING
> > > 17:35:28,892 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1154/2322)
> > > 17:35:28,893 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (1154/2322)
> > > 17:35:28,893 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (1154/2322) [DEPLOYING]
> > > 17:35:28,914 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322)
> switched
> > > to RUNNING
> > > 17:35:28,916 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1429/2322)
> > > 17:35:28,917 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (1429/2322)
> > > 17:35:28,917 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (1429/2322) [DEPLOYING]
> > > 17:35:28,942 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1078/2322)
> > > 17:35:28,942 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (1078/2322)
> > > 17:35:28,942 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (1078/2322) [DEPLOYING]
> > > 17:35:28,943 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322)
> switched
> > > to RUNNING
> > > 17:35:28,955 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322)
> switched
> > > to RUNNING
> > > 17:35:28,959 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (524/2322)
> > > 17:35:28,995 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (524/2322)
> > > 17:35:28,995 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (524/2322) [DEPLOYING]
> > > 17:35:29,000 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2021/2322)
> > > 17:35:29,000 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (2021/2322)
> > > 17:35:29,000 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (2021/2322) [DEPLOYING]
> > > 17:35:29,012 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322)
> switched
> > to
> > > RUNNING
> > > 17:35:29,039 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2022/2322)
> > > 17:35:29,039 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322)
> switched
> > > to RUNNING
> > > 17:35:29,043 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (2022/2322)
> > > 17:35:29,043 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (2022/2322) [DEPLOYING]
> > > 17:35:29,076 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1464/2322)
> > > 17:35:29,081 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (1464/2322)
> > > 17:35:29,081 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (1464/2322) [DEPLOYING]
> > > 17:35:29,095 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322)
> switched
> > > to RUNNING
> > > 17:35:29,108 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1095/2322)
> > > 17:35:29,110 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322)
> switched
> > > to RUNNING
> > > 17:35:29,112 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (1095/2322)
> > > 17:35:29,112 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (1095/2322) [DEPLOYING]
> > > 17:35:29,140 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2306/2322)
> > > 17:35:29,142 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (2306/2322)
> > > 17:35:29,142 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (2306/2322) [DEPLOYING]
> > > 17:35:29,147 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322)
> switched
> > > to RUNNING
> > > 17:35:29,152 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (974/2322)
> > > 17:35:29,153 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322)
> switched
> > > to RUNNING
> > > 17:35:29,155 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (974/2322)
> > > 17:35:29,155 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (974/2322) [DEPLOYING]
> > > 17:35:29,166 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Received
> > > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2305/2322)
> > > 17:35:29,167 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - Loading
> > JAR
> > > files for task CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > > (2305/2322)
> > > 17:35:29,167 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     -
> > Registering
> > > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> > (checksum())
> > > (2305/2322) [DEPLOYING]
> > > 17:35:29,176 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322)
> switched
> > to
> > > RUNNING
> > > 17:35:29,205 INFO
> > > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > > GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322)
> switched
> > > to RUNNING
> > > 17:35:29,417 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 590/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:29,417 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:29,417 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:30,418 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 614/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:30,418 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:30,418 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:31,418 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 634/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:31,418 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:31,419 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:32,419 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 638/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:32,419 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:32,419 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:33,487 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:33,494 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:33,522 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:34,523 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 662/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:34,523 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:34,523 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:35,523 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 670/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:35,524 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:35,524 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:36,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 717/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:36,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:36,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:37,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 737/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:37,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:37,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:38,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:38,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:38,525 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:39,526 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 817/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:39,526 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:39,526 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:40,526 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 832/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:40,526 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:40,526 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:41,527 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 840/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:41,527 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:41,527 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:42,527 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 847/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:42,527 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:42,527 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:43,599 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 450/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:43,599 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:43,599 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:44,599 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 508/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:44,599 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:44,599 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:45,600 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 517/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:45,600 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:45,600 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:46,600 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 528/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:46,600 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:46,600 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:47,663 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 541/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:47,664 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:47,664 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:48,791 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 554/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:48,791 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:48,791 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:49,794 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 562/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:49,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:49,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:50,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 569/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:50,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:50,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:51,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 582/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:51,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:51,795 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:52,796 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 593/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:52,796 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:52,796 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:53,796 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 600/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:53,796 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:53,796 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:54,797 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 604/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:54,797 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:54,797 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:55,797 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 610/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:55,797 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:55,797 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:56,797 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 615/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:56,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:56,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:57,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 624/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:57,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:57,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:58,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 636/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:58,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:58,798 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:35:59,799 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 641/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:35:59,799 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:35:59,799 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:36:00,799 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > > (used/committed/max)]
> > > 17:36:00,799 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:36:00,799 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:36:01,821 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 655/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > (used/committed/max)]
> > > 17:36:01,936 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:36:01,936 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:36:02,937 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 665/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > (used/committed/max)]
> > > 17:36:02,937 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:36:02,937 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > > 17:36:03,944 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > > usage stats: [HEAP: 666/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > > (used/committed/max)]
> > > 17:36:03,950 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              -
> Off-heap
> > > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)],
> [Metaspace:
> > > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > > (used/committed/max)]
> > > 17:36:03,951 INFO
> > > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > >
> > > On Thu, Oct 29, 2015 at 1:55 PM, Till Rohrmann <tr...@apache.org>
> > wrote:
> > >
> > >> What does the log of the failed TaskManager 10.0.88.140 say?
> > >>
> > >> On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <co...@greghogan.com>
> wrote:
> > >>
> > >>> I removed the use of numactl but left in starting two TaskManagers
> and
> > am
> > >>> still seeing TaskManagers crash.
> > >>> From the JobManager log:
> > >>>
> > >>> 17:36:06,412 WARN
> > >>> akka.remote.ReliableDeliverySupervisor                        -
> > >> Association
> > >>> with remote system [akka.tcp://flink@10.0.88.140:45742] has failed,
> > >>> address
> > >>> is now gated for [5000] ms. Reason is: [Disassociated].
> > >>> 17:36:06,567 INFO
> > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
> > >>> (cac9927a8568c2ad79439262a91478af) switched from RUNNING to FAILED
> > >>> 17:36:06,572 INFO
> > >>> org.apache.flink.runtime.jobmanager.JobManager                -
> Status
> > of
> > >>> job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct 29
> > >> 17:34:48
> > >>> UTC 2015) changed to FAILING.
> > >>> java.lang.Exception: The data preparation for task 'CHAIN GroupReduce
> > >>> (Compute scores) -> FlatMap (checksum())' , caused an error: Error
> > >>> obtaining the sorted input: Thread 'SortMerger Reading Thread'
> > terminated
> > >>> due to an exception: Connection unexpectedly closed by remote task
> > >> manager
> > >>> 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the
> > remote
> > >>> task manager was lost.
> > >>>        at
> > >>> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
> > >>>        at
> > >>>
> org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
> > >>>        at
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> > >>>        at java.lang.Thread.run(Thread.java:745)
> > >>> Caused by: java.lang.RuntimeException: Error obtaining the sorted
> > input:
> > >>> Thread 'SortMerger Reading Thread' terminated due to an exception:
> > >>> Connection unexpectedly closed by remote task manager
> 'ip-10-0-88-140/
> > >>> 10.0.88.140:58558'. This might indicate that the remote task manager
> > was
> > >>> lost.
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
> > >>>        at
> > >>>
> > >>
> >
> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
> > >>>        at
> > >>> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
> > >>>        ... 3 more
> > >>> Caused by: java.io.IOException: Thread 'SortMerger Reading Thread'
> > >>> terminated due to an exception: Connection unexpectedly closed by
> > remote
> > >>> task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate
> > >> that
> > >>> the remote task manager was lost.
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
> > >>> Caused by:
> > >>>
> > >>>
> > >>
> >
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
> > >>> Connection unexpectedly closed by remote task manager
> 'ip-10-0-88-140/
> > >>> 10.0.88.140:58558'. This might indicate that the remote task manager
> > was
> > >>> lost.
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
> > >>>        at
> io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> > >>>        at
> > >>>
> > >>>
> > >>
> >
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
> > >>>        at java.lang.Thread.run(Thread.java:745)
> > >>> 17:36:06,587 INFO
> > >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> > >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
> > >>> (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to FAILED
> > >>>
> > >>>
> > >>> On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org>
> > wrote:
> > >>>
> > >>>> Hi Greg!
> > >>>>
> > >>>> Interesting... When you say the TaskManagers are dropping, are the
> > >>>> TaskManager processes crashing, or are they loosing connection to
> the
> > >>>> JobManager?
> > >>>>
> > >>>> Greetings,
> > >>>> Stephan
> > >>>>
> > >>>>
> > >>>> On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com>
> > >> wrote:
> > >>>>
> > >>>>> I recently discovered that AWS uses NUMA for its largest nodes. An
> > >>>> example
> > >>>>> c4.8xlarge:
> > >>>>>
> > >>>>> $ numactl --hardware
> > >>>>> available: 2 nodes (0-1)
> > >>>>> node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> > >>>>> node 0 size: 29813 MB
> > >>>>> node 0 free: 24537 MB
> > >>>>> node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
> > >>>>> node 1 size: 30574 MB
> > >>>>> node 1 free: 22757 MB
> > >>>>> node distances:
> > >>>>> node   0   1
> > >>>>>  0:  10  20
> > >>>>>  1:  20  10
> > >>>>>
> > >>>>> I discovered yesterday that Flink performed ~20-30% faster on large
> > >>>>> datasets by running two NUMA-constrained TaskManagers per node. The
> > >>>>> JobManager node ran a single TaskManager. Resources were divided in
> > >>> half
> > >>>>> relative to running a single TaskManager.
> > >>>>>
> > >>>>> The changes from the tail of /bin/taskmanager.sh:
> > >>>>>
> > >>>>> -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager
> > >> "${args[@]}"
> > >>>>> +numactl --membind=0 --cpunodebind=0
> > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > >>>>> +numactl --membind=1 --cpunodebind=1
> > >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > >>>>> $STARTSTOP taskmanager "${args[@]}"
> > >>>>>
> > >>>>> After reverting this change the system is again stable. I had not
> > >>>>> experienced issues using numactl when running 16 nodes.
> > >>>>>
> > >>>>> Greg
> > >>>>>
> > >>>>
> > >>>
> > >>
> >
> >
>

Re: Diagnosing TaskManager disappearance

Posted by Stephan Ewen <se...@apache.org>.
Thanks for sharing the logs, Greg!

Okay, so the TaskManager does not crash, but the Remote Failure Detector of
Akka marks the connection between JobManager and TaskManager as broken.

The TaskManager is not doing much GC, so it is not a long JVM freeze that
causes hearbeats to time out...

I am wondering at this point whether this is an issue in Akka, specifically
the remote death watch that we use to let the JobManager recognize
disconnected TaskManagers.

One thing you could try is actually to comment out the line where the
JobManager starts the death watch for the TaskManager and see if they can
still successfully exchange messages (tasks finished, find inputs,
schedule) and the program completes. That would indicate that the Akka
Death Watch is flawed and that we should probably do our own heartbeats
instead.

Greetings,
Stephan


On Thu, Oct 29, 2015 at 11:44 AM, Aljoscha Krettek <al...@apache.org>
wrote:

> Could it be a problem that there are two TaskManagers running per machine?
>
> > On 29 Oct 2015, at 19:04, Greg Hogan <co...@greghogan.com> wrote:
> >
> > I have memory logging enabled. Tail of TaskManager log on 10.0.88.140:
> >
> > 17:35:26,415 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:27,415 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 576/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > (used/committed/max)]
> > 17:35:27,415 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:27,415 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:28,012 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (938/2322)
> > 17:35:28,015 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (938/2322)
> > 17:35:28,016 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (938/2322) [DEPLOYING]
> > 17:35:28,065 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322) switched
> to
> > RUNNING
> > 17:35:28,100 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2304/2322)
> > 17:35:28,116 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2304/2322)
> > 17:35:28,116 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (2304/2322) [DEPLOYING]
> > 17:35:28,132 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322) switched
> > to RUNNING
> > 17:35:28,255 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (939/2322)
> > 17:35:28,263 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (939/2322)
> > 17:35:28,263 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (939/2322) [DEPLOYING]
> > 17:35:28,304 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2062/2322)
> > 17:35:28,311 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2062/2322)
> > 17:35:28,311 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (2062/2322) [DEPLOYING]
> > 17:35:28,323 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322) switched
> to
> > RUNNING
> > 17:35:28,386 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322) switched
> > to RUNNING
> > 17:35:28,396 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1775/2322)
> > 17:35:28,401 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1775/2322)
> > 17:35:28,402 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (1775/2322) [DEPLOYING]
> > 17:35:28,416 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 56/58/-1 MB
> > (used/committed/max)]
> > 17:35:28,416 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:28,416 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:28,419 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322) switched
> > to RUNNING
> > 17:35:28,475 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2158/2322)
> > 17:35:28,475 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2158/2322)
> > 17:35:28,476 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (2158/2322) [DEPLOYING]
> > 17:35:28,509 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1463/2322)
> > 17:35:28,860 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1463/2322)
> > 17:35:28,861 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (1463/2322) [DEPLOYING]
> > 17:35:28,862 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322) switched
> > to RUNNING
> > 17:35:28,878 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322) switched
> > to RUNNING
> > 17:35:28,892 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1154/2322)
> > 17:35:28,893 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1154/2322)
> > 17:35:28,893 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (1154/2322) [DEPLOYING]
> > 17:35:28,914 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322) switched
> > to RUNNING
> > 17:35:28,916 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1429/2322)
> > 17:35:28,917 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1429/2322)
> > 17:35:28,917 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (1429/2322) [DEPLOYING]
> > 17:35:28,942 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1078/2322)
> > 17:35:28,942 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1078/2322)
> > 17:35:28,942 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (1078/2322) [DEPLOYING]
> > 17:35:28,943 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322) switched
> > to RUNNING
> > 17:35:28,955 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322) switched
> > to RUNNING
> > 17:35:28,959 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (524/2322)
> > 17:35:28,995 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (524/2322)
> > 17:35:28,995 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (524/2322) [DEPLOYING]
> > 17:35:29,000 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2021/2322)
> > 17:35:29,000 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2021/2322)
> > 17:35:29,000 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (2021/2322) [DEPLOYING]
> > 17:35:29,012 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322) switched
> to
> > RUNNING
> > 17:35:29,039 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2022/2322)
> > 17:35:29,039 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322) switched
> > to RUNNING
> > 17:35:29,043 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2022/2322)
> > 17:35:29,043 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (2022/2322) [DEPLOYING]
> > 17:35:29,076 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1464/2322)
> > 17:35:29,081 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1464/2322)
> > 17:35:29,081 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (1464/2322) [DEPLOYING]
> > 17:35:29,095 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322) switched
> > to RUNNING
> > 17:35:29,108 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1095/2322)
> > 17:35:29,110 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322) switched
> > to RUNNING
> > 17:35:29,112 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (1095/2322)
> > 17:35:29,112 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (1095/2322) [DEPLOYING]
> > 17:35:29,140 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2306/2322)
> > 17:35:29,142 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2306/2322)
> > 17:35:29,142 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (2306/2322) [DEPLOYING]
> > 17:35:29,147 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322) switched
> > to RUNNING
> > 17:35:29,152 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (974/2322)
> > 17:35:29,153 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322) switched
> > to RUNNING
> > 17:35:29,155 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (974/2322)
> > 17:35:29,155 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (974/2322) [DEPLOYING]
> > 17:35:29,166 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Received
> > task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2305/2322)
> > 17:35:29,167 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - Loading
> JAR
> > files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> > (2305/2322)
> > 17:35:29,167 INFO
> > org.apache.flink.runtime.taskmanager.Task                     -
> Registering
> > task at network: CHAIN GroupReduce (Compute scores) -> FlatMap
> (checksum())
> > (2305/2322) [DEPLOYING]
> > 17:35:29,176 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322) switched
> to
> > RUNNING
> > 17:35:29,205 INFO
> > org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322) switched
> > to RUNNING
> > 17:35:29,417 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 590/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > (used/committed/max)]
> > 17:35:29,417 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:29,417 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:30,418 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 614/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > (used/committed/max)]
> > 17:35:30,418 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:30,418 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:31,418 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 634/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > (used/committed/max)]
> > 17:35:31,418 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:31,419 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:32,419 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 638/1917/1917 MB, NON HEAP: 57/59/-1 MB
> > (used/committed/max)]
> > 17:35:32,419 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:32,419 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:33,487 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:33,494 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:33,522 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:34,523 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 662/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:34,523 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:34,523 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:35,523 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 670/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:35,524 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:35,524 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:36,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 717/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:36,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:36,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:37,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 737/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:37,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:37,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:38,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:38,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:38,525 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:39,526 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 817/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:39,526 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:39,526 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:40,526 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 832/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:40,526 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:40,526 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:41,527 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 840/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:41,527 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:41,527 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:42,527 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 847/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:42,527 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:42,527 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:43,599 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 450/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:43,599 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:43,599 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:44,599 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 508/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:44,599 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:44,599 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:45,600 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 517/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:45,600 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:45,600 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:46,600 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 528/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:46,600 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:46,600 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:47,663 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 541/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:47,664 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:47,664 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:48,791 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 554/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:48,791 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:48,791 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:49,794 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 562/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:49,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:49,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:50,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 569/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:50,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:50,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:51,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 582/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:51,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:51,795 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:52,796 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 593/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:52,796 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:52,796 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:53,796 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 600/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:53,796 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:53,796 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:54,797 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 604/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:54,797 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:54,797 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:55,797 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 610/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:55,797 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:55,797 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:56,797 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 615/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:56,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:56,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:57,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 624/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:57,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:57,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:58,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 636/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:58,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:58,798 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:35:59,799 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 641/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:35:59,799 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:35:59,799 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:36:00,799 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> > (used/committed/max)]
> > 17:36:00,799 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:36:00,799 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:36:01,821 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 655/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > (used/committed/max)]
> > 17:36:01,936 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:36:01,936 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:36:02,937 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 665/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > (used/committed/max)]
> > 17:36:02,937 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:36:02,937 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> > 17:36:03,944 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> > usage stats: [HEAP: 666/1917/1917 MB, NON HEAP: 58/60/-1 MB
> > (used/committed/max)]
> > 17:36:03,950 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> > pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> > 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> > (used/committed/max)]
> > 17:36:03,951 INFO
> > org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> > collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> > MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> >
> > On Thu, Oct 29, 2015 at 1:55 PM, Till Rohrmann <tr...@apache.org>
> wrote:
> >
> >> What does the log of the failed TaskManager 10.0.88.140 say?
> >>
> >> On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <co...@greghogan.com> wrote:
> >>
> >>> I removed the use of numactl but left in starting two TaskManagers and
> am
> >>> still seeing TaskManagers crash.
> >>> From the JobManager log:
> >>>
> >>> 17:36:06,412 WARN
> >>> akka.remote.ReliableDeliverySupervisor                        -
> >> Association
> >>> with remote system [akka.tcp://flink@10.0.88.140:45742] has failed,
> >>> address
> >>> is now gated for [5000] ms. Reason is: [Disassociated].
> >>> 17:36:06,567 INFO
> >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
> >>> (cac9927a8568c2ad79439262a91478af) switched from RUNNING to FAILED
> >>> 17:36:06,572 INFO
> >>> org.apache.flink.runtime.jobmanager.JobManager                - Status
> of
> >>> job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct 29
> >> 17:34:48
> >>> UTC 2015) changed to FAILING.
> >>> java.lang.Exception: The data preparation for task 'CHAIN GroupReduce
> >>> (Compute scores) -> FlatMap (checksum())' , caused an error: Error
> >>> obtaining the sorted input: Thread 'SortMerger Reading Thread'
> terminated
> >>> due to an exception: Connection unexpectedly closed by remote task
> >> manager
> >>> 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the
> remote
> >>> task manager was lost.
> >>>        at
> >>> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
> >>>        at
> >>> org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
> >>>        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> >>>        at java.lang.Thread.run(Thread.java:745)
> >>> Caused by: java.lang.RuntimeException: Error obtaining the sorted
> input:
> >>> Thread 'SortMerger Reading Thread' terminated due to an exception:
> >>> Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
> >>> 10.0.88.140:58558'. This might indicate that the remote task manager
> was
> >>> lost.
> >>>        at
> >>>
> >>>
> >>
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
> >>>        at
> >>>
> >>
> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
> >>>        at
> >>>
> >>>
> >>
> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
> >>>        at
> >>> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
> >>>        ... 3 more
> >>> Caused by: java.io.IOException: Thread 'SortMerger Reading Thread'
> >>> terminated due to an exception: Connection unexpectedly closed by
> remote
> >>> task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate
> >> that
> >>> the remote task manager was lost.
> >>>        at
> >>>
> >>>
> >>
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
> >>> Caused by:
> >>>
> >>>
> >>
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
> >>> Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
> >>> 10.0.88.140:58558'. This might indicate that the remote task manager
> was
> >>> lost.
> >>>        at
> >>>
> >>>
> >>
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> >>>        at
> >>>
> >>>
> >>
> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
> >>>        at
> >>>
> >>>
> >>
> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
> >>>        at
> >>>
> >>>
> >>
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
> >>>        at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> >>>        at
> >>>
> >>>
> >>
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
> >>>        at java.lang.Thread.run(Thread.java:745)
> >>> 17:36:06,587 INFO
> >>> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> >>> GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
> >>> (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to FAILED
> >>>
> >>>
> >>> On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org>
> wrote:
> >>>
> >>>> Hi Greg!
> >>>>
> >>>> Interesting... When you say the TaskManagers are dropping, are the
> >>>> TaskManager processes crashing, or are they loosing connection to the
> >>>> JobManager?
> >>>>
> >>>> Greetings,
> >>>> Stephan
> >>>>
> >>>>
> >>>> On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com>
> >> wrote:
> >>>>
> >>>>> I recently discovered that AWS uses NUMA for its largest nodes. An
> >>>> example
> >>>>> c4.8xlarge:
> >>>>>
> >>>>> $ numactl --hardware
> >>>>> available: 2 nodes (0-1)
> >>>>> node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> >>>>> node 0 size: 29813 MB
> >>>>> node 0 free: 24537 MB
> >>>>> node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
> >>>>> node 1 size: 30574 MB
> >>>>> node 1 free: 22757 MB
> >>>>> node distances:
> >>>>> node   0   1
> >>>>>  0:  10  20
> >>>>>  1:  20  10
> >>>>>
> >>>>> I discovered yesterday that Flink performed ~20-30% faster on large
> >>>>> datasets by running two NUMA-constrained TaskManagers per node. The
> >>>>> JobManager node ran a single TaskManager. Resources were divided in
> >>> half
> >>>>> relative to running a single TaskManager.
> >>>>>
> >>>>> The changes from the tail of /bin/taskmanager.sh:
> >>>>>
> >>>>> -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager
> >> "${args[@]}"
> >>>>> +numactl --membind=0 --cpunodebind=0
> >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> >>>>> $STARTSTOP taskmanager "${args[@]}"
> >>>>> +numactl --membind=1 --cpunodebind=1
> >> "${FLINK_BIN_DIR}"/flink-daemon.sh
> >>>>> $STARTSTOP taskmanager "${args[@]}"
> >>>>>
> >>>>> After reverting this change the system is again stable. I had not
> >>>>> experienced issues using numactl when running 16 nodes.
> >>>>>
> >>>>> Greg
> >>>>>
> >>>>
> >>>
> >>
>
>

Re: Diagnosing TaskManager disappearance

Posted by Aljoscha Krettek <al...@apache.org>.
Could it be a problem that there are two TaskManagers running per machine?

> On 29 Oct 2015, at 19:04, Greg Hogan <co...@greghogan.com> wrote:
> 
> I have memory logging enabled. Tail of TaskManager log on 10.0.88.140:
> 
> 17:35:26,415 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:27,415 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 576/1917/1917 MB, NON HEAP: 56/58/-1 MB
> (used/committed/max)]
> 17:35:27,415 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:27,415 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:28,012 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322)
> 17:35:28,015 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (938/2322)
> 17:35:28,016 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (938/2322) [DEPLOYING]
> 17:35:28,065 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322) switched to
> RUNNING
> 17:35:28,100 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322)
> 17:35:28,116 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2304/2322)
> 17:35:28,116 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2304/2322) [DEPLOYING]
> 17:35:28,132 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322) switched
> to RUNNING
> 17:35:28,255 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322)
> 17:35:28,263 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (939/2322)
> 17:35:28,263 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (939/2322) [DEPLOYING]
> 17:35:28,304 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322)
> 17:35:28,311 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2062/2322)
> 17:35:28,311 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2062/2322) [DEPLOYING]
> 17:35:28,323 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322) switched to
> RUNNING
> 17:35:28,386 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322) switched
> to RUNNING
> 17:35:28,396 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322)
> 17:35:28,401 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1775/2322)
> 17:35:28,402 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1775/2322) [DEPLOYING]
> 17:35:28,416 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 56/58/-1 MB
> (used/committed/max)]
> 17:35:28,416 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:28,416 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:28,419 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322) switched
> to RUNNING
> 17:35:28,475 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322)
> 17:35:28,475 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2158/2322)
> 17:35:28,476 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2158/2322) [DEPLOYING]
> 17:35:28,509 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322)
> 17:35:28,860 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1463/2322)
> 17:35:28,861 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1463/2322) [DEPLOYING]
> 17:35:28,862 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322) switched
> to RUNNING
> 17:35:28,878 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322) switched
> to RUNNING
> 17:35:28,892 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322)
> 17:35:28,893 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1154/2322)
> 17:35:28,893 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1154/2322) [DEPLOYING]
> 17:35:28,914 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322) switched
> to RUNNING
> 17:35:28,916 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322)
> 17:35:28,917 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1429/2322)
> 17:35:28,917 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1429/2322) [DEPLOYING]
> 17:35:28,942 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322)
> 17:35:28,942 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1078/2322)
> 17:35:28,942 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1078/2322) [DEPLOYING]
> 17:35:28,943 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322) switched
> to RUNNING
> 17:35:28,955 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322) switched
> to RUNNING
> 17:35:28,959 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322)
> 17:35:28,995 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (524/2322)
> 17:35:28,995 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (524/2322) [DEPLOYING]
> 17:35:29,000 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322)
> 17:35:29,000 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2021/2322)
> 17:35:29,000 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2021/2322) [DEPLOYING]
> 17:35:29,012 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322) switched to
> RUNNING
> 17:35:29,039 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322)
> 17:35:29,039 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322) switched
> to RUNNING
> 17:35:29,043 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2022/2322)
> 17:35:29,043 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2022/2322) [DEPLOYING]
> 17:35:29,076 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322)
> 17:35:29,081 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1464/2322)
> 17:35:29,081 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1464/2322) [DEPLOYING]
> 17:35:29,095 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322) switched
> to RUNNING
> 17:35:29,108 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322)
> 17:35:29,110 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322) switched
> to RUNNING
> 17:35:29,112 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1095/2322)
> 17:35:29,112 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (1095/2322) [DEPLOYING]
> 17:35:29,140 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322)
> 17:35:29,142 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2306/2322)
> 17:35:29,142 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2306/2322) [DEPLOYING]
> 17:35:29,147 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322) switched
> to RUNNING
> 17:35:29,152 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322)
> 17:35:29,153 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322) switched
> to RUNNING
> 17:35:29,155 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (974/2322)
> 17:35:29,155 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (974/2322) [DEPLOYING]
> 17:35:29,166 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Received
> task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322)
> 17:35:29,167 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
> files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2305/2322)
> 17:35:29,167 INFO
> org.apache.flink.runtime.taskmanager.Task                     - Registering
> task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
> (2305/2322) [DEPLOYING]
> 17:35:29,176 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322) switched to
> RUNNING
> 17:35:29,205 INFO
> org.apache.flink.runtime.taskmanager.Task                     - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322) switched
> to RUNNING
> 17:35:29,417 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 590/1917/1917 MB, NON HEAP: 57/59/-1 MB
> (used/committed/max)]
> 17:35:29,417 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:29,417 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:30,418 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 614/1917/1917 MB, NON HEAP: 57/59/-1 MB
> (used/committed/max)]
> 17:35:30,418 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:30,418 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:31,418 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 634/1917/1917 MB, NON HEAP: 57/59/-1 MB
> (used/committed/max)]
> 17:35:31,418 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:31,419 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:32,419 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 638/1917/1917 MB, NON HEAP: 57/59/-1 MB
> (used/committed/max)]
> 17:35:32,419 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:32,419 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:33,487 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:33,494 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:33,522 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:34,523 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 662/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:34,523 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:34,523 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:35,523 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 670/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:35,524 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:35,524 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:36,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 717/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:36,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:36,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:37,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 737/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:37,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:37,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:38,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:38,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:38,525 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:39,526 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 817/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:39,526 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:39,526 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:40,526 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 832/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:40,526 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:40,526 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:41,527 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 840/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:41,527 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:41,527 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:42,527 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 847/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:42,527 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:42,527 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:43,599 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 450/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:43,599 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:43,599 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:44,599 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 508/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:44,599 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:44,599 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:45,600 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 517/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:45,600 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:45,600 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:46,600 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 528/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:46,600 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:46,600 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:47,663 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 541/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:47,664 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:47,664 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:48,791 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 554/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:48,791 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:48,791 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:49,794 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 562/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:49,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:49,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:50,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 569/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:50,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:50,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:51,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 582/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:51,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:51,795 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:52,796 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 593/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:52,796 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:52,796 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:53,796 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 600/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:53,796 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:53,796 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:54,797 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 604/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:54,797 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:54,797 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:55,797 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 610/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:55,797 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:55,797 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:56,797 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 615/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:56,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:56,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:57,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 624/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:57,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:57,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:58,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 636/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:58,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:58,798 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:35:59,799 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 641/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:35:59,799 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:35:59,799 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:36:00,799 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
> (used/committed/max)]
> 17:36:00,799 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:36:00,799 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:36:01,821 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 655/1917/1917 MB, NON HEAP: 58/60/-1 MB
> (used/committed/max)]
> 17:36:01,936 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:36:01,936 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:36:02,937 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 665/1917/1917 MB, NON HEAP: 58/60/-1 MB
> (used/committed/max)]
> 17:36:02,937 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:36:02,937 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 17:36:03,944 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Memory
> usage stats: [HEAP: 666/1917/1917 MB, NON HEAP: 58/60/-1 MB
> (used/committed/max)]
> 17:36:03,950 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
> pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
> 34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
> (used/committed/max)]
> 17:36:03,951 INFO
> org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
> collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
> MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
> 
> On Thu, Oct 29, 2015 at 1:55 PM, Till Rohrmann <tr...@apache.org> wrote:
> 
>> What does the log of the failed TaskManager 10.0.88.140 say?
>> 
>> On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <co...@greghogan.com> wrote:
>> 
>>> I removed the use of numactl but left in starting two TaskManagers and am
>>> still seeing TaskManagers crash.
>>> From the JobManager log:
>>> 
>>> 17:36:06,412 WARN
>>> akka.remote.ReliableDeliverySupervisor                        -
>> Association
>>> with remote system [akka.tcp://flink@10.0.88.140:45742] has failed,
>>> address
>>> is now gated for [5000] ms. Reason is: [Disassociated].
>>> 17:36:06,567 INFO
>>> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
>>> GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
>>> (cac9927a8568c2ad79439262a91478af) switched from RUNNING to FAILED
>>> 17:36:06,572 INFO
>>> org.apache.flink.runtime.jobmanager.JobManager                - Status of
>>> job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct 29
>> 17:34:48
>>> UTC 2015) changed to FAILING.
>>> java.lang.Exception: The data preparation for task 'CHAIN GroupReduce
>>> (Compute scores) -> FlatMap (checksum())' , caused an error: Error
>>> obtaining the sorted input: Thread 'SortMerger Reading Thread' terminated
>>> due to an exception: Connection unexpectedly closed by remote task
>> manager
>>> 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the remote
>>> task manager was lost.
>>>        at
>>> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
>>>        at
>>> org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
>>>        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>>>        at java.lang.Thread.run(Thread.java:745)
>>> Caused by: java.lang.RuntimeException: Error obtaining the sorted input:
>>> Thread 'SortMerger Reading Thread' terminated due to an exception:
>>> Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
>>> 10.0.88.140:58558'. This might indicate that the remote task manager was
>>> lost.
>>>        at
>>> 
>>> 
>> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
>>>        at
>>> 
>> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
>>>        at
>>> 
>>> 
>> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
>>>        at
>>> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
>>>        ... 3 more
>>> Caused by: java.io.IOException: Thread 'SortMerger Reading Thread'
>>> terminated due to an exception: Connection unexpectedly closed by remote
>>> task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate
>> that
>>> the remote task manager was lost.
>>>        at
>>> 
>>> 
>> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
>>> Caused by:
>>> 
>>> 
>> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
>>> Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
>>> 10.0.88.140:58558'. This might indicate that the remote task manager was
>>> lost.
>>>        at
>>> 
>>> 
>> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
>>>        at
>>> 
>>> 
>> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
>>>        at
>>> 
>>> 
>> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
>>>        at
>>> 
>>> 
>> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
>>>        at
>>> 
>>> 
>> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
>>>        at
>>> 
>>> 
>> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
>>>        at
>>> 
>>> 
>> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
>>>        at
>>> 
>>> 
>> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
>>>        at
>>> 
>>> 
>> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
>>>        at
>>> 
>>> 
>> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
>>>        at
>>> 
>>> 
>> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
>>>        at
>>> 
>>> 
>> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
>>>        at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
>>>        at
>>> 
>>> 
>> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
>>>        at java.lang.Thread.run(Thread.java:745)
>>> 17:36:06,587 INFO
>>> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
>>> GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
>>> (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to FAILED
>>> 
>>> 
>>> On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org> wrote:
>>> 
>>>> Hi Greg!
>>>> 
>>>> Interesting... When you say the TaskManagers are dropping, are the
>>>> TaskManager processes crashing, or are they loosing connection to the
>>>> JobManager?
>>>> 
>>>> Greetings,
>>>> Stephan
>>>> 
>>>> 
>>>> On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com>
>> wrote:
>>>> 
>>>>> I recently discovered that AWS uses NUMA for its largest nodes. An
>>>> example
>>>>> c4.8xlarge:
>>>>> 
>>>>> $ numactl --hardware
>>>>> available: 2 nodes (0-1)
>>>>> node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
>>>>> node 0 size: 29813 MB
>>>>> node 0 free: 24537 MB
>>>>> node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
>>>>> node 1 size: 30574 MB
>>>>> node 1 free: 22757 MB
>>>>> node distances:
>>>>> node   0   1
>>>>>  0:  10  20
>>>>>  1:  20  10
>>>>> 
>>>>> I discovered yesterday that Flink performed ~20-30% faster on large
>>>>> datasets by running two NUMA-constrained TaskManagers per node. The
>>>>> JobManager node ran a single TaskManager. Resources were divided in
>>> half
>>>>> relative to running a single TaskManager.
>>>>> 
>>>>> The changes from the tail of /bin/taskmanager.sh:
>>>>> 
>>>>> -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager
>> "${args[@]}"
>>>>> +numactl --membind=0 --cpunodebind=0
>> "${FLINK_BIN_DIR}"/flink-daemon.sh
>>>>> $STARTSTOP taskmanager "${args[@]}"
>>>>> +numactl --membind=1 --cpunodebind=1
>> "${FLINK_BIN_DIR}"/flink-daemon.sh
>>>>> $STARTSTOP taskmanager "${args[@]}"
>>>>> 
>>>>> After reverting this change the system is again stable. I had not
>>>>> experienced issues using numactl when running 16 nodes.
>>>>> 
>>>>> Greg
>>>>> 
>>>> 
>>> 
>> 


Re: Diagnosing TaskManager disappearance

Posted by Greg Hogan <co...@greghogan.com>.
I have memory logging enabled. Tail of TaskManager log on 10.0.88.140:

17:35:26,415 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:27,415 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 576/1917/1917 MB, NON HEAP: 56/58/-1 MB
(used/committed/max)]
17:35:27,415 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:27,415 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:28,012 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322)
17:35:28,015 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(938/2322)
17:35:28,016 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(938/2322) [DEPLOYING]
17:35:28,065 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (938/2322) switched to
RUNNING
17:35:28,100 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322)
17:35:28,116 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2304/2322)
17:35:28,116 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2304/2322) [DEPLOYING]
17:35:28,132 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (2304/2322) switched
to RUNNING
17:35:28,255 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322)
17:35:28,263 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(939/2322)
17:35:28,263 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(939/2322) [DEPLOYING]
17:35:28,304 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322)
17:35:28,311 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2062/2322)
17:35:28,311 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2062/2322) [DEPLOYING]
17:35:28,323 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (939/2322) switched to
RUNNING
17:35:28,386 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (2062/2322) switched
to RUNNING
17:35:28,396 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322)
17:35:28,401 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1775/2322)
17:35:28,402 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1775/2322) [DEPLOYING]
17:35:28,416 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 56/58/-1 MB
(used/committed/max)]
17:35:28,416 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:28,416 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 341, GC COUNT: 3], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:28,419 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (1775/2322) switched
to RUNNING
17:35:28,475 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322)
17:35:28,475 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2158/2322)
17:35:28,476 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2158/2322) [DEPLOYING]
17:35:28,509 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322)
17:35:28,860 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1463/2322)
17:35:28,861 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1463/2322) [DEPLOYING]
17:35:28,862 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (2158/2322) switched
to RUNNING
17:35:28,878 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (1463/2322) switched
to RUNNING
17:35:28,892 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322)
17:35:28,893 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1154/2322)
17:35:28,893 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1154/2322) [DEPLOYING]
17:35:28,914 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (1154/2322) switched
to RUNNING
17:35:28,916 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322)
17:35:28,917 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1429/2322)
17:35:28,917 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1429/2322) [DEPLOYING]
17:35:28,942 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322)
17:35:28,942 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1078/2322)
17:35:28,942 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1078/2322) [DEPLOYING]
17:35:28,943 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (1429/2322) switched
to RUNNING
17:35:28,955 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (1078/2322) switched
to RUNNING
17:35:28,959 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322)
17:35:28,995 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(524/2322)
17:35:28,995 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(524/2322) [DEPLOYING]
17:35:29,000 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322)
17:35:29,000 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2021/2322)
17:35:29,000 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2021/2322) [DEPLOYING]
17:35:29,012 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (524/2322) switched to
RUNNING
17:35:29,039 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322)
17:35:29,039 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (2021/2322) switched
to RUNNING
17:35:29,043 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2022/2322)
17:35:29,043 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2022/2322) [DEPLOYING]
17:35:29,076 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322)
17:35:29,081 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1464/2322)
17:35:29,081 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1464/2322) [DEPLOYING]
17:35:29,095 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (2022/2322) switched
to RUNNING
17:35:29,108 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322)
17:35:29,110 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (1464/2322) switched
to RUNNING
17:35:29,112 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1095/2322)
17:35:29,112 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(1095/2322) [DEPLOYING]
17:35:29,140 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322)
17:35:29,142 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2306/2322)
17:35:29,142 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2306/2322) [DEPLOYING]
17:35:29,147 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (1095/2322) switched
to RUNNING
17:35:29,152 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322)
17:35:29,153 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (2306/2322) switched
to RUNNING
17:35:29,155 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(974/2322)
17:35:29,155 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(974/2322) [DEPLOYING]
17:35:29,166 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Received
task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322)
17:35:29,167 INFO
org.apache.flink.runtime.taskmanager.Task                     - Loading JAR
files for task CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2305/2322)
17:35:29,167 INFO
org.apache.flink.runtime.taskmanager.Task                     - Registering
task at network: CHAIN GroupReduce (Compute scores) -> FlatMap (checksum())
(2305/2322) [DEPLOYING]
17:35:29,176 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (974/2322) switched to
RUNNING
17:35:29,205 INFO
org.apache.flink.runtime.taskmanager.Task                     - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (2305/2322) switched
to RUNNING
17:35:29,417 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 590/1917/1917 MB, NON HEAP: 57/59/-1 MB
(used/committed/max)]
17:35:29,417 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:29,417 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:30,418 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 614/1917/1917 MB, NON HEAP: 57/59/-1 MB
(used/committed/max)]
17:35:30,418 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 18/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:30,418 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:31,418 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 634/1917/1917 MB, NON HEAP: 57/59/-1 MB
(used/committed/max)]
17:35:31,418 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:31,419 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:32,419 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 638/1917/1917 MB, NON HEAP: 57/59/-1 MB
(used/committed/max)]
17:35:32,419 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:32,419 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:33,487 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:33,494 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:33,522 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:34,523 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 662/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:34,523 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:34,523 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:35,523 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 670/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:35,524 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:35,524 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:36,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 717/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:36,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:36,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:37,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 737/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:37,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/19/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:37,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:38,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 747/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:38,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:38,525 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:39,526 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 817/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:39,526 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:39,526 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:40,526 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 832/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:40,526 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:40,526 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:41,527 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 840/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:41,527 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:41,527 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:42,527 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 847/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:42,527 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:42,527 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 689, GC COUNT: 4], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:43,599 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 450/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:43,599 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:43,599 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:44,599 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 508/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:44,599 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:44,599 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:45,600 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 517/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:45,600 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:45,600 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:46,600 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 528/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:46,600 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:46,600 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:47,663 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 541/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:47,664 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:47,664 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:48,791 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 554/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:48,791 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:48,791 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:49,794 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 562/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:49,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:49,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:50,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 569/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:50,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:50,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:51,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 582/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:51,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:51,795 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:52,796 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 593/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:52,796 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:52,796 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:53,796 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 600/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:53,796 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:53,796 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:54,797 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 604/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:54,797 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:54,797 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:55,797 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 610/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:55,797 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:55,797 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:56,797 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 615/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:56,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:56,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:57,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 624/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:57,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:57,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:58,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 636/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:58,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:58,798 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:35:59,799 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 641/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:35:59,799 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:35:59,799 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:36:00,799 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 648/1917/1917 MB, NON HEAP: 58/59/-1 MB
(used/committed/max)]
17:36:00,799 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/34/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:36:00,799 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:36:01,821 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 655/1917/1917 MB, NON HEAP: 58/60/-1 MB
(used/committed/max)]
17:36:01,936 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:36:01,936 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:36:02,937 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 665/1917/1917 MB, NON HEAP: 58/60/-1 MB
(used/committed/max)]
17:36:02,937 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:36:02,937 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]
17:36:03,944 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Memory
usage stats: [HEAP: 666/1917/1917 MB, NON HEAP: 58/60/-1 MB
(used/committed/max)]
17:36:03,950 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Off-heap
pool stats: [Code Cache: 19/20/240 MB (used/committed/max)], [Metaspace:
34/35/-1 MB (used/committed/max)], [Compressed Class Space: 4/4/1024 MB
(used/committed/max)]
17:36:03,951 INFO
org.apache.flink.runtime.taskmanager.TaskManager              - Garbage
collector stats: [PS Scavenge, GC TIME (ms): 797, GC COUNT: 5], [PS
MarkSweep, GC TIME (ms): 974, GC COUNT: 1]

On Thu, Oct 29, 2015 at 1:55 PM, Till Rohrmann <tr...@apache.org> wrote:

> What does the log of the failed TaskManager 10.0.88.140 say?
>
> On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <co...@greghogan.com> wrote:
>
> > I removed the use of numactl but left in starting two TaskManagers and am
> > still seeing TaskManagers crash.
> > From the JobManager log:
> >
> > 17:36:06,412 WARN
> > akka.remote.ReliableDeliverySupervisor                        -
> Association
> > with remote system [akka.tcp://flink@10.0.88.140:45742] has failed,
> > address
> > is now gated for [5000] ms. Reason is: [Disassociated].
> > 17:36:06,567 INFO
> > org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
> > (cac9927a8568c2ad79439262a91478af) switched from RUNNING to FAILED
> > 17:36:06,572 INFO
> > org.apache.flink.runtime.jobmanager.JobManager                - Status of
> > job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct 29
> 17:34:48
> > UTC 2015) changed to FAILING.
> > java.lang.Exception: The data preparation for task 'CHAIN GroupReduce
> > (Compute scores) -> FlatMap (checksum())' , caused an error: Error
> > obtaining the sorted input: Thread 'SortMerger Reading Thread' terminated
> > due to an exception: Connection unexpectedly closed by remote task
> manager
> > 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the remote
> > task manager was lost.
> >         at
> > org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
> >         at
> > org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
> >         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
> >         at java.lang.Thread.run(Thread.java:745)
> > Caused by: java.lang.RuntimeException: Error obtaining the sorted input:
> > Thread 'SortMerger Reading Thread' terminated due to an exception:
> > Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
> > 10.0.88.140:58558'. This might indicate that the remote task manager was
> > lost.
> >         at
> >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
> >         at
> >
> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
> >         at
> >
> >
> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
> >         at
> > org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
> >         ... 3 more
> > Caused by: java.io.IOException: Thread 'SortMerger Reading Thread'
> > terminated due to an exception: Connection unexpectedly closed by remote
> > task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate
> that
> > the remote task manager was lost.
> >         at
> >
> >
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
> > Caused by:
> >
> >
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
> > Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
> > 10.0.88.140:58558'. This might indicate that the remote task manager was
> > lost.
> >         at
> >
> >
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
> >         at
> >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> >         at
> >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> >         at
> >
> >
> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
> >         at
> >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> >         at
> >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> >         at
> >
> >
> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
> >         at
> >
> >
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
> >         at
> >
> >
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
> >         at
> >
> >
> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
> >         at
> >
> >
> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
> >         at
> >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
> >         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
> >         at
> >
> >
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
> >         at java.lang.Thread.run(Thread.java:745)
> > 17:36:06,587 INFO
> > org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> > GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
> > (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to FAILED
> >
> >
> > On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org> wrote:
> >
> > > Hi Greg!
> > >
> > > Interesting... When you say the TaskManagers are dropping, are the
> > > TaskManager processes crashing, or are they loosing connection to the
> > > JobManager?
> > >
> > > Greetings,
> > > Stephan
> > >
> > >
> > > On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com>
> wrote:
> > >
> > > > I recently discovered that AWS uses NUMA for its largest nodes. An
> > > example
> > > > c4.8xlarge:
> > > >
> > > > $ numactl --hardware
> > > > available: 2 nodes (0-1)
> > > > node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> > > > node 0 size: 29813 MB
> > > > node 0 free: 24537 MB
> > > > node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
> > > > node 1 size: 30574 MB
> > > > node 1 free: 22757 MB
> > > > node distances:
> > > > node   0   1
> > > >   0:  10  20
> > > >   1:  20  10
> > > >
> > > > I discovered yesterday that Flink performed ~20-30% faster on large
> > > > datasets by running two NUMA-constrained TaskManagers per node. The
> > > > JobManager node ran a single TaskManager. Resources were divided in
> > half
> > > > relative to running a single TaskManager.
> > > >
> > > > The changes from the tail of /bin/taskmanager.sh:
> > > >
> > > > -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager
> "${args[@]}"
> > > > +numactl --membind=0 --cpunodebind=0
> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > > $STARTSTOP taskmanager "${args[@]}"
> > > > +numactl --membind=1 --cpunodebind=1
> "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > > $STARTSTOP taskmanager "${args[@]}"
> > > >
> > > > After reverting this change the system is again stable. I had not
> > > > experienced issues using numactl when running 16 nodes.
> > > >
> > > > Greg
> > > >
> > >
> >
>

Re: Diagnosing TaskManager disappearance

Posted by Till Rohrmann <tr...@apache.org>.
What does the log of the failed TaskManager 10.0.88.140 say?

On Thu, Oct 29, 2015 at 6:44 PM, Greg Hogan <co...@greghogan.com> wrote:

> I removed the use of numactl but left in starting two TaskManagers and am
> still seeing TaskManagers crash.
> From the JobManager log:
>
> 17:36:06,412 WARN
> akka.remote.ReliableDeliverySupervisor                        - Association
> with remote system [akka.tcp://flink@10.0.88.140:45742] has failed,
> address
> is now gated for [5000] ms. Reason is: [Disassociated].
> 17:36:06,567 INFO
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
> (cac9927a8568c2ad79439262a91478af) switched from RUNNING to FAILED
> 17:36:06,572 INFO
> org.apache.flink.runtime.jobmanager.JobManager                - Status of
> job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct 29 17:34:48
> UTC 2015) changed to FAILING.
> java.lang.Exception: The data preparation for task 'CHAIN GroupReduce
> (Compute scores) -> FlatMap (checksum())' , caused an error: Error
> obtaining the sorted input: Thread 'SortMerger Reading Thread' terminated
> due to an exception: Connection unexpectedly closed by remote task manager
> 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the remote
> task manager was lost.
>         at
> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
>         at
> org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
>         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.RuntimeException: Error obtaining the sorted input:
> Thread 'SortMerger Reading Thread' terminated due to an exception:
> Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
> 10.0.88.140:58558'. This might indicate that the remote task manager was
> lost.
>         at
>
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
>         at
> org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
>         at
>
> org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
>         at
> org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
>         ... 3 more
> Caused by: java.io.IOException: Thread 'SortMerger Reading Thread'
> terminated due to an exception: Connection unexpectedly closed by remote
> task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that
> the remote task manager was lost.
>         at
>
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
> Caused by:
>
> org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
> Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
> 10.0.88.140:58558'. This might indicate that the remote task manager was
> lost.
>         at
>
> org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
>         at
>
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
>         at
>
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
>         at
>
> io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
>         at
>
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
>         at
>
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
>         at
>
> io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
>         at
>
> io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
>         at
>
> io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
>         at
>
> io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
>         at
>
> io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
>         at
>
> io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
>         at
>
> io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
>         at java.lang.Thread.run(Thread.java:745)
> 17:36:06,587 INFO
> org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
> GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
> (d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to FAILED
>
>
> On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org> wrote:
>
> > Hi Greg!
> >
> > Interesting... When you say the TaskManagers are dropping, are the
> > TaskManager processes crashing, or are they loosing connection to the
> > JobManager?
> >
> > Greetings,
> > Stephan
> >
> >
> > On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com> wrote:
> >
> > > I recently discovered that AWS uses NUMA for its largest nodes. An
> > example
> > > c4.8xlarge:
> > >
> > > $ numactl --hardware
> > > available: 2 nodes (0-1)
> > > node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> > > node 0 size: 29813 MB
> > > node 0 free: 24537 MB
> > > node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
> > > node 1 size: 30574 MB
> > > node 1 free: 22757 MB
> > > node distances:
> > > node   0   1
> > >   0:  10  20
> > >   1:  20  10
> > >
> > > I discovered yesterday that Flink performed ~20-30% faster on large
> > > datasets by running two NUMA-constrained TaskManagers per node. The
> > > JobManager node ran a single TaskManager. Resources were divided in
> half
> > > relative to running a single TaskManager.
> > >
> > > The changes from the tail of /bin/taskmanager.sh:
> > >
> > > -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager "${args[@]}"
> > > +numactl --membind=0 --cpunodebind=0 "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > $STARTSTOP taskmanager "${args[@]}"
> > > +numactl --membind=1 --cpunodebind=1 "${FLINK_BIN_DIR}"/flink-daemon.sh
> > > $STARTSTOP taskmanager "${args[@]}"
> > >
> > > After reverting this change the system is again stable. I had not
> > > experienced issues using numactl when running 16 nodes.
> > >
> > > Greg
> > >
> >
>

Re: Diagnosing TaskManager disappearance

Posted by Greg Hogan <co...@greghogan.com>.
I removed the use of numactl but left in starting two TaskManagers and am
still seeing TaskManagers crash.
>From the JobManager log:

17:36:06,412 WARN
akka.remote.ReliableDeliverySupervisor                        - Association
with remote system [akka.tcp://flink@10.0.88.140:45742] has failed, address
is now gated for [5000] ms. Reason is: [Disassociated].
17:36:06,567 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (370/2322)
(cac9927a8568c2ad79439262a91478af) switched from RUNNING to FAILED
17:36:06,572 INFO
org.apache.flink.runtime.jobmanager.JobManager                - Status of
job 14d946015fd7b35eb801ea6fee5af9e4 (Flink Java Job at Thu Oct 29 17:34:48
UTC 2015) changed to FAILING.
java.lang.Exception: The data preparation for task 'CHAIN GroupReduce
(Compute scores) -> FlatMap (checksum())' , caused an error: Error
obtaining the sorted input: Thread 'SortMerger Reading Thread' terminated
due to an exception: Connection unexpectedly closed by remote task manager
'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that the remote
task manager was lost.
        at
org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:465)
        at
org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:354)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
        at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.RuntimeException: Error obtaining the sorted input:
Thread 'SortMerger Reading Thread' terminated due to an exception:
Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
10.0.88.140:58558'. This might indicate that the remote task manager was
lost.
        at
org.apache.flink.runtime.operators.sort.UnilateralSortMerger.getIterator(UnilateralSortMerger.java:619)
        at
org.apache.flink.runtime.operators.BatchTask.getInput(BatchTask.java:1089)
        at
org.apache.flink.runtime.operators.GroupReduceDriver.prepare(GroupReduceDriver.java:94)
        at
org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:459)
        ... 3 more
Caused by: java.io.IOException: Thread 'SortMerger Reading Thread'
terminated due to an exception: Connection unexpectedly closed by remote
task manager 'ip-10-0-88-140/10.0.88.140:58558'. This might indicate that
the remote task manager was lost.
        at
org.apache.flink.runtime.operators.sort.UnilateralSortMerger$ThreadBase.run(UnilateralSortMerger.java:800)
Caused by:
org.apache.flink.runtime.io.network.netty.exception.RemoteTransportException:
Connection unexpectedly closed by remote task manager 'ip-10-0-88-140/
10.0.88.140:58558'. This might indicate that the remote task manager was
lost.
        at
org.apache.flink.runtime.io.network.netty.PartitionRequestClientHandler.channelInactive(PartitionRequestClientHandler.java:119)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
        at
io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
        at
io.netty.channel.ChannelInboundHandlerAdapter.channelInactive(ChannelInboundHandlerAdapter.java:75)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
        at
io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
        at
io.netty.handler.codec.ByteToMessageDecoder.channelInactive(ByteToMessageDecoder.java:306)
        at
io.netty.channel.AbstractChannelHandlerContext.invokeChannelInactive(AbstractChannelHandlerContext.java:208)
        at
io.netty.channel.AbstractChannelHandlerContext.fireChannelInactive(AbstractChannelHandlerContext.java:194)
        at
io.netty.channel.DefaultChannelPipeline.fireChannelInactive(DefaultChannelPipeline.java:828)
        at
io.netty.channel.AbstractChannel$AbstractUnsafe$7.run(AbstractChannel.java:621)
        at
io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:358)
        at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
        at
io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:112)
        at java.lang.Thread.run(Thread.java:745)
17:36:06,587 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN
GroupReduce (Compute scores) -> FlatMap (checksum()) (367/2322)
(d63c681a18b8164bc24936df1ecb159b) switched from RUNNING to FAILED


On Thu, Oct 29, 2015 at 1:00 PM, Stephan Ewen <se...@apache.org> wrote:

> Hi Greg!
>
> Interesting... When you say the TaskManagers are dropping, are the
> TaskManager processes crashing, or are they loosing connection to the
> JobManager?
>
> Greetings,
> Stephan
>
>
> On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com> wrote:
>
> > I recently discovered that AWS uses NUMA for its largest nodes. An
> example
> > c4.8xlarge:
> >
> > $ numactl --hardware
> > available: 2 nodes (0-1)
> > node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> > node 0 size: 29813 MB
> > node 0 free: 24537 MB
> > node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
> > node 1 size: 30574 MB
> > node 1 free: 22757 MB
> > node distances:
> > node   0   1
> >   0:  10  20
> >   1:  20  10
> >
> > I discovered yesterday that Flink performed ~20-30% faster on large
> > datasets by running two NUMA-constrained TaskManagers per node. The
> > JobManager node ran a single TaskManager. Resources were divided in half
> > relative to running a single TaskManager.
> >
> > The changes from the tail of /bin/taskmanager.sh:
> >
> > -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager "${args[@]}"
> > +numactl --membind=0 --cpunodebind=0 "${FLINK_BIN_DIR}"/flink-daemon.sh
> > $STARTSTOP taskmanager "${args[@]}"
> > +numactl --membind=1 --cpunodebind=1 "${FLINK_BIN_DIR}"/flink-daemon.sh
> > $STARTSTOP taskmanager "${args[@]}"
> >
> > After reverting this change the system is again stable. I had not
> > experienced issues using numactl when running 16 nodes.
> >
> > Greg
> >
>

Re: Diagnosing TaskManager disappearance

Posted by Stephan Ewen <se...@apache.org>.
Hi Greg!

Interesting... When you say the TaskManagers are dropping, are the
TaskManager processes crashing, or are they loosing connection to the
JobManager?

Greetings,
Stephan


On Thu, Oct 29, 2015 at 9:56 AM, Greg Hogan <co...@greghogan.com> wrote:

> I recently discovered that AWS uses NUMA for its largest nodes. An example
> c4.8xlarge:
>
> $ numactl --hardware
> available: 2 nodes (0-1)
> node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
> node 0 size: 29813 MB
> node 0 free: 24537 MB
> node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
> node 1 size: 30574 MB
> node 1 free: 22757 MB
> node distances:
> node   0   1
>   0:  10  20
>   1:  20  10
>
> I discovered yesterday that Flink performed ~20-30% faster on large
> datasets by running two NUMA-constrained TaskManagers per node. The
> JobManager node ran a single TaskManager. Resources were divided in half
> relative to running a single TaskManager.
>
> The changes from the tail of /bin/taskmanager.sh:
>
> -"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager "${args[@]}"
> +numactl --membind=0 --cpunodebind=0 "${FLINK_BIN_DIR}"/flink-daemon.sh
> $STARTSTOP taskmanager "${args[@]}"
> +numactl --membind=1 --cpunodebind=1 "${FLINK_BIN_DIR}"/flink-daemon.sh
> $STARTSTOP taskmanager "${args[@]}"
>
> After reverting this change the system is again stable. I had not
> experienced issues using numactl when running 16 nodes.
>
> Greg
>

Re: Diagnosing TaskManager disappearance

Posted by Greg Hogan <co...@greghogan.com>.
I recently discovered that AWS uses NUMA for its largest nodes. An example
c4.8xlarge:

$ numactl --hardware
available: 2 nodes (0-1)
node 0 cpus: 0 1 2 3 4 5 6 7 8 18 19 20 21 22 23 24 25 26
node 0 size: 29813 MB
node 0 free: 24537 MB
node 1 cpus: 9 10 11 12 13 14 15 16 17 27 28 29 30 31 32 33 34 35
node 1 size: 30574 MB
node 1 free: 22757 MB
node distances:
node   0   1
  0:  10  20
  1:  20  10

I discovered yesterday that Flink performed ~20-30% faster on large
datasets by running two NUMA-constrained TaskManagers per node. The
JobManager node ran a single TaskManager. Resources were divided in half
relative to running a single TaskManager.

The changes from the tail of /bin/taskmanager.sh:

-"${FLINK_BIN_DIR}"/flink-daemon.sh $STARTSTOP taskmanager "${args[@]}"
+numactl --membind=0 --cpunodebind=0 "${FLINK_BIN_DIR}"/flink-daemon.sh
$STARTSTOP taskmanager "${args[@]}"
+numactl --membind=1 --cpunodebind=1 "${FLINK_BIN_DIR}"/flink-daemon.sh
$STARTSTOP taskmanager "${args[@]}"

After reverting this change the system is again stable. I had not
experienced issues using numactl when running 16 nodes.

Greg

Re: Diagnosing TaskManager disappearance

Posted by Maximilian Michels <mx...@apache.org>.
Hi Greg,

Thanks for reporting. You wrote you didn't see any output in the .out files
of the task managers. What about the .log files of these instances?

Where and when did you produce the thread dump you included?

Thanks,
Max

On Thu, Oct 29, 2015 at 1:46 PM, Greg Hogan <co...@greghogan.com> wrote:

> I am testing again on a 64 node cluster (the JobManager is running fine
> having reduced some operator's parallelism and fixed the string conversion
> performance).
>
> I am seeing TaskManagers drop like flies every other job or so. I am not
> seeing any output in the .out log files corresponding to the crashed
> TaskManagers.
>
> Below is the stack trace from a java.hprof heap dump.
>
> How should I be debugging this?
>
> Thanks,
> Greg
>
>
> Threads at the heap dump:
>
> Unknown thread
>
>
> "Memory Logger" daemon prio=1 tid=119 TIMED_WAITING
> at java.lang.Thread.<init>(Thread.java:507)
> at
>
> org.apache.flink.runtime.taskmanager.MemoryLogger.<init>(MemoryLogger.java:67)
> at
>
> org.apache.flink.runtime.taskmanager.TaskManager$.runTaskManager(TaskManager.scala:1494)
> at
>
> org.apache.flink.runtime.taskmanager.TaskManager$.selectNetworkInterfaceAndRunTaskManager(TaskManager.scala:1330)
>
>
> "Flink Netty Server (59693) Thread 0" daemon prio=5 tid=193 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:674)
> at
>
> java.util.concurrent.Executors$DefaultThreadFactory.newThread(Executors.java:613)
> at
>
> org.apache.flink.shaded.com.google.common.util.concurrent.ThreadFactoryBuilder$1.newThread(ThreadFactoryBuilder.java:162)
> at
>
> io.netty.util.concurrent.SingleThreadEventExecutor.<init>(SingleThreadEventExecutor.java:106)
>
>
> "flink-akka.remote.default-remote-dispatcher-6" daemon prio=5 tid=30
> TIMED_WAITING
> at java.lang.Thread.<init>(Thread.java:507)
> at
>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
> at
>
> akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)
>
>
> "flink-akka.actor.default-dispatcher-4" daemon prio=5 tid=28 WAITING
> at java.lang.Thread.<init>(Thread.java:507)
> at
>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
> at
>
> akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)
>
>
> "flink-akka.remote.default-remote-dispatcher-5" daemon prio=5 tid=29
> WAITING
> at java.lang.Thread.<init>(Thread.java:507)
> at
>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
> at
>
> akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)
>
>
> "flink-akka.actor.default-dispatcher-2" daemon prio=5 tid=26 WAITING
> at java.lang.Thread.<init>(Thread.java:507)
> at
>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
> at
>
> akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)
>
>
> "SIGTERM handler" daemon prio=9 tid=268 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:547)
> at sun.misc.Signal.dispatch(Signal.java:216)
>
>
> "HPROF gc_finish watcher" daemon prio=10 tid=5 RUNNABLE
>
>
> "Reference Handler" daemon prio=10 tid=2 WAITING
>
>
> "main" prio=5 tid=1 WAITING
>
>
> "Signal Dispatcher" daemon prio=9 tid=4 RUNNABLE
>
>
> "Finalizer" daemon prio=8 tid=3 WAITING
>
>
> "flink-akka.actor.default-dispatcher-3" daemon prio=5 tid=27 TIMED_WAITING
> at java.lang.Thread.<init>(Thread.java:507)
> at
>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.<init>(ForkJoinWorkerThread.java:48)
> at
>
> akka.dispatch.MonitorableThreadFactory$AkkaForkJoinWorkerThread.<init>(ThreadPoolBuilder.scala:164)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:187)
>
>
> "New I/O worker #1" daemon prio=5 tid=31 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at
>
> java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
> at
>
> java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)
>
>
> "flink-scheduler-1" daemon prio=5 tid=25 TIMED_WAITING
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at akka.actor.LightArrayRevolverScheduler.<init>(Scheduler.scala:337)
> at
>
> sun.reflect.NativeConstructorAccessorImpl.newInstance0(NativeConstructorAccessorImpl.java)
>
>
> "New I/O worker #2" daemon prio=5 tid=32 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at
>
> java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
> at
>
> java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)
>
>
> "Hashed wheel timer #1" daemon prio=5 tid=33 TIMED_WAITING
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at org.jboss.netty.util.HashedWheelTimer.<init>(HashedWheelTimer.java:226)
>   Local Variable: java.util.ArrayList#502
> at org.jboss.netty.util.HashedWheelTimer.<init>(HashedWheelTimer.java:177)
>   Local Variable: java.lang.String#15234
>
>
> "New I/O boss #3" daemon prio=5 tid=34 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at
>
> java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
> at
>
> java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)
>
>
> "Timer-0" daemon prio=5 tid=267 TIMED_WAITING
> at java.lang.Thread.<init>(Thread.java:444)
> at java.util.TimerThread.<init>(Timer.java:499)
> at java.util.Timer.<init>(Timer.java:101)
> at java.util.Timer.<init>(Timer.java:146)
>
>
> "New I/O worker #4" daemon prio=5 tid=35 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at
>
> java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
> at
>
> java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)
>
>
> "New I/O worker #5" daemon prio=5 tid=36 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at
>
> java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
> at
>
> java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)
>
>
> "New I/O server boss #6" daemon prio=5 tid=37 RUNNABLE
> at java.lang.Thread.<init>(Thread.java:547)
> at
>
> akka.dispatch.MonitorableThreadFactory.newThread(ThreadPoolBuilder.scala:193)
> at
>
> java.util.concurrent.ThreadPoolExecutor$Worker.<init>(ThreadPoolExecutor.java:612)
> at
>
> java.util.concurrent.ThreadPoolExecutor.addWorker(ThreadPoolExecutor.java:925)
>