To verify wether GC is a problem you can enable logging of memory usage of
the JVM via taskmanager.debug.memory.startLogThread: true. The interval of
the logging is configured via taskmanager.debug.memory.logIntervalMs.
​

On Thu, Oct 15, 2015 at 12:00 PM, Matthias J. Sax <mj...@apache.org> wrote:

> The problem is reproducible (it happens on each run).
>
> I doubt that GC is an issue here (at least from an UDF point of view),
> because I read the file once and keep a String object for each line.
> This objects are kept to the very end; the UDF does not release them
> until it returns from "run()" method.
>
> The file is also small (10K) so main memory consumption should be a
> problem either.
>
> Of course, the UDF emits on a very high rate and I am not sure if Flink
> creates a lots of short living object (Streaming does not reuse object
> yet...) such that GC might become an issue here?
>
>
> -Matthias
>
>
>
> On 10/15/2015 11:44 AM, Stephan Ewen wrote:
> > From what the logs show, the TaskManager does not send pings any more
> for a
> > long time and is then considered failed and the tasks running on that
> > TaskManager are considered failed as well. So far, nothing unusual...
> >
> > Question is, why is it considered failed? Is this a reproducible problem?
> > Or a one time occurrence? Does the user code accumulate so many objects
> to
> > put very heavy pressure on the GC?
> >
> > Stephan
> >
> >
> > On Wed, Oct 14, 2015 at 10:47 PM, Matthias J. Sax <mj...@apache.org>
> wrote:
> >
> >> One thing I forgot the add. I also have a Storm-WordCount job (build via
> >> FlinkTopologyBuilder) that uses the same
> >> "buffer-file-and-emit-over-and-over-again-pattern" in a spout. This job
> >> run just fine and stops regularly after 5 minutes.
> >>
> >> -Matthias
> >>
> >>
> >> On 10/14/2015 10:42 PM, Matthias J. Sax wrote:
> >>> No. See log below.
> >>>
> >>> Btw: the job is not cleaned up properly. Some task remain in state
> >>> "Canceling".
> >>>
> >>> The program I execute is "Streaming WordCount" example with my own
> >>> source function. This custom source (see below), reads a local (small)
> >>> file, bufferes each line in an internal buffer, and emits the buffered
> >>> lines over and over again (for 5 minutes). (A copy of the file is
> >>> located on every worker machine of course.) This is of course no common
> >>> patttern; I just do this, because I have only a small file and want the
> >>> job to run longer.
> >>>
> >>> As the job is running for multiple minutes until it fails, I assume
> that
> >>> this uncommon pattern should actually not cauase any problem. I just
> >>> report it, to make sure it is not the problem (hope you can confirm).
> >>>
> >>> -Matthias
> >>>
> >>>
> >>>>      private static final class SimpleFileSource implements
> >> ParallelSourceFunction<String> {
> >>>>              private static final long serialVersionUID =
> >> -5727198990308179551L;
> >>>>
> >>>>              volatile boolean running = true;
> >>>>              @Override
> >>>>              public void run(
> >>>>
> >>
> org.apache.flink.streaming.api.functions.source.SourceFunction.SourceContext<String>
> >> ctx)
> >>>>                                              throws Exception {
> >>>>
> >>>>                      BufferedReader reader = new BufferedReader(new
> >> FileReader(
> >>>>                                      "/data/mjsax/hamlet.txt"));
> >>>>                      ArrayList<String> buffer = new
> ArrayList<String>();
> >>>>                      int counter = 0;
> >>>>                      String line;
> >>>>
> >>>>                      long start = System.nanoTime();
> >>>>                      while (running && (line = reader.readLine()) !=
> >> null) {
> >>>>                              buffer.add(line);
> >>>>                              ctx.collect(line);
> >>>>                      }
> >>>>                      reader.close();
> >>>>
> >>>>                      while (running && (System.nanoTime() - start) <
> 5L
> >> * 60 * 1000 * 1000 * 1000) {
> >>>>                              ctx.collect(buffer.get(counter));
> >>>>                              counter = ++counter % buffer.size();
> >>>>                      }
> >>>>              }
> >>>>
> >>>>              @Override
> >>>>              public void cancel() {
> >>>>                      running = false;
> >>>>              }
> >>>>      }
> >>>
> >>>
> >>> JobManager log:
> >>>
> >>>> 22:19:16,618 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (2/28) (29bb077aa0772c0c42a21b782d496b7c)
> >> switched from DEPLOYING to RUNNING
> >>>> 22:22:52,082 WARN  akka.remote.RemoteWatcher
> >>          - Detected unreachable: [akka.tcp://flink@192.168.127.32:32994
> ]
> >>>> 22:22:52,083 INFO  org.apache.flink.runtime.jobmanager.JobManager
> >>           - Task manager akka.tcp://
> >> flink@192.168.127.32:32994/user/taskmanager terminated.
> >>>> 22:22:52,084 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (8/28) (b3fb63bdac8f446e8bbaa015f30abfdc)
> >> switched from RUNNING to FAILED
> >>>> 22:22:52,093 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (1/28) (e18d7fb12ea2b5b88dda07e79a6c62ba)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,096 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (2/28) (b5af8ba1031971c07b8655bdf62acdea)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,096 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (3/28) (47d0cbfb8472a50ac005305dfab91202)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,096 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (4/28) (39144b5a2aa58be4667bdc38c50602b5)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,096 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (5/28) (c6dcb3a5d755b162dbe13f32de406a1d)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,097 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (6/28) (09d9649f2e9c83dbbe3d1b253651f136)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,097 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (7/28) (f82fda5a32c0f96acda4185ade9e985f)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,097 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (9/28) (dc9d2905ae42980a25c8870a1d89c056)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,097 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (10/28) (2e9682428b32fca6fca787e1a6b71359)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,097 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (11/28) (bae79dd7f2b9346c7d168db8143c0bd4)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,098 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (12/28) (642c76fc78faafcb023bce9cec041e9d)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,098 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (13/28) (1cf21c3abc8f91f82b5fb6175181baeb)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,098 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (14/28) (97fe45f48434c568652a9f9cd885b2da)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,098 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (15/28) (b8833cdda3fc205865ef611d176b33ea)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,098 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (16/28) (157a107c35ac994503cca577389a4f80)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,099 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (17/28) (509d1dacb4448b741aaf170009a31364)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,099 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (18/28) (d6bce51dd6d00b6b2f7e59ffcc733242)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,099 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (19/28) (dc6931d39b054b761cfed7345d683c29)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,099 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (20/28) (0f288fa6cc5910235e32564c1740670f)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,099 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (21/28) (80b811c2787d34c67941686e3c2be74a)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,100 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (22/28) (c70d69252e7b3ff6a537a2b09180d818)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,100 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (23/28) (29154b2299dd080e041ca8a349d3b418)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,100 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (24/28) (f56827e4079567adb23406c44f765dcf)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,100 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (25/28) (cd6f30e3a69b02367a6ddd097f9fb825)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,100 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (26/28) (c4b3af71e26718513752595a6cbdcee5)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,101 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (27/28) (b152ab3209ca6c68c6f20213ccb48f40)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,101 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (28/28) (efcee09b90ca372bffa4d086ce0b104a)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,101 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (1/28) (b2189b3d40cb4fa24083ffefd43ffc4f)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,101 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (2/28) (29bb077aa0772c0c42a21b782d496b7c)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,101 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (3/28) (2eb046c3d88cb7e6deccb5ac9d3a3704)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,102 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (4/28) (63f6696e53f08dc420bb51a10a092531)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,102 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (5/28) (bd9c6fc6d89e64dd3bef9c21065dfc87)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,102 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (6/28) (b675c869f60552cc04c3842d297a2529)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,102 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (7/28) (3f56dfd2ff3fe595555be6a372bf7c7b)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,102 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (8/28) (94d29535cf2559e5fe6dc13b8a5e3f08)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,103 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (9/28) (5f99a88b2041719d202f20bbc5dd2ca6)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,103 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (10/28) (7c1a75ec9794ae9197ac3e64ecbd4ad8)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,103 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (11/28) (c110922406d6f58cd34c80abf1ace972)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,103 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (12/28) (2f991c806e12dd9715da532f35d3cd83)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,103 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (13/28) (4b59d9e7f5102f7a5d1c09e49ffca66e)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,104 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (14/28) (6b966d3247685983a73b439666fe315a)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,104 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (15/28) (b7f040b44a87db85c41b7f8e1808aecc)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,104 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (16/28) (1a42f51a278bef23c497da8d34a28e91)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,104 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (17/28) (b3d86978bc1bf04e2cab2700fb6975e0)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,104 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (18/28) (a44cb0a419524a0b7e04d476285b03c0)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,104 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (19/28) (0d2b87be2ec0eb7961c05a4cf7baf7a7)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,105 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (20/28) (ebd6220fc5d98912811c19aa9fe29e63)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,105 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (21/28) (6cd95e1bb7c22e310967eab4149fa93d)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,105 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (22/28) (ddc9b7f1c122e2d36632c8f8e886d35b)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,105 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (23/28) (0e3b8cc17d0169a12142532dd800f966)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,105 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (24/28) (4ac383b01d8c5a2d395612202d1f876c)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,106 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (25/28) (bf542fee30071b7cfa63bae514af6106)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,106 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (26/28) (e60b40069c1c82c44318e053eacfa7bd)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,106 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (27/28) (35d1ce78a6f61a75af41c7b6d9c403a9)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,106 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (28/28) (de58a786dbd2557dcc7c54bf2710525b)
> >> switched from RUNNING to CANCELING
> >>>> 22:22:52,107 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (16/28) (1a42f51a278bef23c497da8d34a28e91)
> >> switched from CANCELING to FAILED
> >>>> 22:22:52,107 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (28/28) (efcee09b90ca372bffa4d086ce0b104a)
> >> switched from CANCELING to FAILED
> >>>> 22:22:52,108 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (15/28) (b7f040b44a87db85c41b7f8e1808aecc)
> >> switched from CANCELING to FAILED
> >>>> 22:22:52,108 INFO  org.apache.flink.runtime.instance.InstanceManager
> >>          - Unregistered task manager akka.tcp://
> >> flink@192.168.127.32:32994/user/taskmanager. Number of registered task
> >> managers 19. Number of available slots 228.
> >>>> 22:22:52,109 INFO  org.apache.flink.runtime.jobmanager.JobManager
> >>           - Status of job 11b9636948e49d96965051902b0aee1f (Streaming
> >> WordCount) changed to FAILING.
> >>>> java.lang.Exception: The slot in which the task was executed has been
> >> released. Probably loss of TaskManager 8175fe04b5cad804b197fbd1f8e2a599
> @
> >> dbis32 - 12 slots - URL: akka.tcp://
> >> flink@192.168.127.32:32994/user/taskmanager
> >>>>         at
> >>
> org.apache.flink.runtime.instance.SimpleSlot.releaseSlot(SimpleSlot.java:151)
> >>>>         at
> >>
> org.apache.flink.runtime.instance.SlotSharingGroupAssignment.releaseSharedSlot(SlotSharingGroupAssignment.java:547)
> >>>>         at
> >>
> org.apache.flink.runtime.instance.SharedSlot.releaseSlot(SharedSlot.java:119)
> >>>>         at
> >> org.apache.flink.runtime.instance.Instance.markDead(Instance.java:156)
> >>>>         at
> >>
> org.apache.flink.runtime.instance.InstanceManager.unregisterTaskManager(InstanceManager.java:215)
> >>>>         at
> >>
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$handleMessage$1.applyOrElse(JobManager.scala:565)
> >>>>         at
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
> >>>>         at
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
> >>>>         at
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
> >>>>         at
> >>
> org.apache.flink.runtime.LeaderSessionMessageFilter$$anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:44)
> >>>>         at
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
> >>>>         at
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
> >>>>         at
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
> >>>>         at
> >> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:33)
> >>>>         at
> >> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:28)
> >>>>         at
> >> scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118)
> >>>>         at
> >>
> org.apache.flink.runtime.LogMessages$$anon$1.applyOrElse(LogMessages.scala:28)
> >>>>         at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
> >>>>         at
> >>
> org.apache.flink.runtime.jobmanager.JobManager.aroundReceive(JobManager.scala:103)
> >>>>         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> >>>>         at
> >>
> akka.actor.dungeon.DeathWatch$class.receivedTerminated(DeathWatch.scala:46)
> >>>>         at
> akka.actor.ActorCell.receivedTerminated(ActorCell.scala:369)
> >>>>         at
> akka.actor.ActorCell.autoReceiveMessage(ActorCell.scala:501)
> >>>>         at akka.actor.ActorCell.invoke(ActorCell.scala:486)
> >>>>         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:254)
> >>>>         at akka.dispatch.Mailbox.run(Mailbox.scala:221)
> >>>>         at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
> >>>>         at
> >> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> >>>>         at
> >>
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> >>>>         at
> >> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> >>>>         at
> >>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> >>>> 22:22:52,132 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (15/28) (b8833cdda3fc205865ef611d176b33ea)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,132 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (11/28) (bae79dd7f2b9346c7d168db8143c0bd4)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,133 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (9/28) (dc9d2905ae42980a25c8870a1d89c056)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,133 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (18/28) (d6bce51dd6d00b6b2f7e59ffcc733242)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,134 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (6/28) (09d9649f2e9c83dbbe3d1b253651f136)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,135 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (17/28) (509d1dacb4448b741aaf170009a31364)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,135 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (26/28) (c4b3af71e26718513752595a6cbdcee5)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,135 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (5/28) (c6dcb3a5d755b162dbe13f32de406a1d)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,136 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (14/28) (97fe45f48434c568652a9f9cd885b2da)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,136 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (25/28) (cd6f30e3a69b02367a6ddd097f9fb825)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,137 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (13/28) (1cf21c3abc8f91f82b5fb6175181baeb)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,137 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (10/28) (2e9682428b32fca6fca787e1a6b71359)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,137 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (19/28) (dc6931d39b054b761cfed7345d683c29)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,139 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (7/28) (f82fda5a32c0f96acda4185ade9e985f)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,139 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (4/28) (39144b5a2aa58be4667bdc38c50602b5)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,139 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (2/28) (b5af8ba1031971c07b8655bdf62acdea)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,139 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (24/28) (f56827e4079567adb23406c44f765dcf)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,140 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (22/28) (c70d69252e7b3ff6a537a2b09180d818)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,140 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (3/28) (47d0cbfb8472a50ac005305dfab91202)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,140 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (27/28) (b152ab3209ca6c68c6f20213ccb48f40)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,141 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (23/28) (29154b2299dd080e041ca8a349d3b418)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,141 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (20/28) (0f288fa6cc5910235e32564c1740670f)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,142 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (12/28) (642c76fc78faafcb023bce9cec041e9d)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:52,142 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (16/28) (157a107c35ac994503cca577389a4f80)
> >> switched from CANCELING to CANCELED
> >>>> 22:22:57,003 INFO  Remoting
> >>           - Quarantined address [akka.tcp://flink@192.168.127.32:32994]
> >> is still unreachable or has not been restarted. Keeping it quarantined.
> >>>> 22:23:01,078 WARN  akka.remote.RemoteWatcher
> >>          - Detected unreachable: [akka.tcp://flink@192.168.127.62:35242
> ]
> >>>> 22:23:01,079 INFO  org.apache.flink.runtime.jobmanager.JobManager
> >>           - Task manager akka.tcp://
> >> flink@192.168.127.62:35242/user/taskmanager terminated.
> >>>> 22:23:01,079 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (1/28) (b2189b3d40cb4fa24083ffefd43ffc4f)
> >> switched from CANCELING to FAILED
> >>>> 22:23:01,081 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (21/28) (80b811c2787d34c67941686e3c2be74a)
> >> switched from CANCELING to FAILED
> >>>> 22:23:01,082 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Keyed
> >> Aggregation -> Sink: Unnamed (2/28) (29bb077aa0772c0c42a21b782d496b7c)
> >> switched from CANCELING to FAILED
> >>>> 22:23:01,083 INFO
> >> org.apache.flink.runtime.executiongraph.ExecutionGraph        - Source:
> >> Custom Source -> Flat Map (1/28) (e18d7fb12ea2b5b88dda07e79a6c62ba)
> >> switched from CANCELING to FAILED
> >>>> 22:23:01,084 INFO  org.apache.flink.runtime.instance.InstanceManager
> >>          - Unregistered task manager akka.tcp://
> >> flink@192.168.127.62:35242/user/taskmanager. Number of registered task
> >> managers 18. Number of available slots 216.
> >>>
> >>>
> >>>
> >>>
> >>> On 10/11/2015 11:54 PM, Stephan Ewen wrote:
> >>>> Can you see is there is anything unusual in the JobManager logs?
> >>>> Am 11.10.2015 18:56 schrieb "Matthias J. Sax" <mj...@apache.org>:
> >>>>
> >>>>> Hi,
> >>>>>
> >>>>> I was just playing arround with Flink. After submitting my job, it
> runs
> >>>>> for multiple minutes, until I get the following Exception in one if
> the
> >>>>> TaskManager logs and the job fails.
> >>>>>
> >>>>> I have no clue what's going on...
> >>>>>
> >>>>> -Matthias
> >>>>>
> >>>>>
> >>>>>> 18:43:23,567 WARN  akka.remote.RemoteWatcher
> >>>>>          - Detected unreachable: [akka.tcp://
> flink@192.168.127.11:6123
> >> ]
> >>>>>> 18:43:23,864 WARN  akka.remote.ReliableDeliverySupervisor
> >>>>>         - Association with remote system [akka.tcp://
> >>>>> flink@192.168.127.11:6123] has failed, address is now gated for
> [5000]
> >>>>> ms. Reason is: [Disassociated].
> >>>>>> 18:43:23,866 INFO  org.apache.flink.runtime.taskmanager.TaskManager
> >>>>>         - TaskManager akka://flink/user/taskmanager disconnects from
> >>>>> JobManager akka.tcp://flink@192.168.127.11:6123/user/jobmanager:
> >>>>> JobManager is no longer reachable
> >>>>>> 18:43:23,867 INFO  org.apache.flink.runtime.taskmanager.TaskManager
> >>>>>         - Cancelling all computations and discarding all cached data.
> >>>>>> 18:43:23,870 INFO  org.apache.flink.runtime.taskmanager.Task
> >>>>>          - Attempting to fail task externally Keyed Aggregation ->
> >> Sink:
> >>>>> Unnamed (1/28)
> >>>>>> 18:43:23,870 INFO  org.apache.flink.runtime.taskmanager.Task
> >>>>>          - Keyed Aggregation -> Sink: Unnamed (1/28) switched to
> FAILED
> >>>>> with exception.
> >>>>>> java.lang.Exception: TaskManager akka://flink/user/taskmanager
> >>>>> disconnects from JobManager akka.tcp://
> >>>>> flink@192.168.127.11:6123/user/jobmanager: JobManager is no longer
> >>>>> reachable
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.taskmanager.TaskManager.handleJobManagerDisconnect(TaskManager.scala:826)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.taskmanager.TaskManager$$anonfun$handleMessage$1.applyOrElse(TaskManager.scala:297)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.LeaderSessionMessageFilter$$anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:44)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
> >>>>>>         at
> >>>>>
> >> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:33)
> >>>>>>         at
> >>>>>
> >> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:28)
> >>>>>>         at
> >>>>> scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.LogMessages$$anon$1.applyOrElse(LogMessages.scala:28)
> >>>>>>         at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.taskmanager.TaskManager.aroundReceive(TaskManager.scala:119)
> >>>>>>         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> >>>>>>         at
> >>>>>
> >>
> akka.actor.dungeon.DeathWatch$class.receivedTerminated(DeathWatch.scala:46)
> >>>>>>         at
> >> akka.actor.ActorCell.receivedTerminated(ActorCell.scala:369)
> >>>>>>         at
> >> akka.actor.ActorCell.autoReceiveMessage(ActorCell.scala:501)
> >>>>>>         at akka.actor.ActorCell.invoke(ActorCell.scala:486)
> >>>>>>         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:254)
> >>>>>>         at akka.dispatch.Mailbox.run(Mailbox.scala:221)
> >>>>>>         at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
> >>>>>>         at
> >>>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> >>>>>>         at
> >>>>>
> >>
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> >>>>>>         at
> >>>>>
> >> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> >>>>>>         at
> >>>>>
> >>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> >>>>>> 18:43:23,879 INFO  org.apache.flink.runtime.taskmanager.Task
> >>>>>          - Triggering cancellation of task code Keyed Aggregation ->
> >> Sink:
> >>>>> Unnamed (1/28) (4b1f979a60c30f84ee60553730a4e99a).
> >>>>>> 18:43:23,880 INFO  org.apache.flink.runtime.taskmanager.Task
> >>>>>          - Attempting to fail task externally Source: Custom Source
> ->
> >> Flat
> >>>>> Map (1/28)
> >>>>>> 18:43:23,880 INFO  org.apache.flink.runtime.taskmanager.Task
> >>>>>          - Source: Custom Source -> Flat Map (1/28) switched to
> FAILED
> >> with
> >>>>> exception.
> >>>>>> java.lang.Exception: TaskManager akka://flink/user/taskmanager
> >>>>> disconnects from JobManager akka.tcp://
> >>>>> flink@192.168.127.11:6123/user/jobmanager: JobManager is no longer
> >>>>> reachable
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.taskmanager.TaskManager.handleJobManagerDisconnect(TaskManager.scala:826)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.taskmanager.TaskManager$$anonfun$handleMessage$1.applyOrElse(TaskManager.scala:297)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.LeaderSessionMessageFilter$$anonfun$receive$1.applyOrElse(LeaderSessionMessageFilter.scala:44)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply$mcVL$sp(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:33)
> >>>>>>         at
> >>>>>
> >>
> scala.runtime.AbstractPartialFunction$mcVL$sp.apply(AbstractPartialFunction.scala:25)
> >>>>>>         at
> >>>>>
> >> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:33)
> >>>>>>         at
> >>>>>
> >> org.apache.flink.runtime.LogMessages$$anon$1.apply(LogMessages.scala:28)
> >>>>>>         at
> >>>>> scala.PartialFunction$class.applyOrElse(PartialFunction.scala:118)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.LogMessages$$anon$1.applyOrElse(LogMessages.scala:28)
> >>>>>>         at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
> >>>>>>         at
> >>>>>
> >>
> org.apache.flink.runtime.taskmanager.TaskManager.aroundReceive(TaskManager.scala:119)
> >>>>>>         at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> >>>>>>         at
> >>>>>
> >>
> akka.actor.dungeon.DeathWatch$class.receivedTerminated(DeathWatch.scala:46)
> >>>>>>         at
> >> akka.actor.ActorCell.receivedTerminated(ActorCell.scala:369)
> >>>>>>         at
> >> akka.actor.ActorCell.autoReceiveMessage(ActorCell.scala:501)
> >>>>>>         at akka.actor.ActorCell.invoke(ActorCell.scala:486)
> >>>>>>         at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:254)
> >>>>>>         at akka.dispatch.Mailbox.run(Mailbox.scala:221)
> >>>>>>         at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
> >>>>>>         at
> >>>>> scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> >>>>>>         at
> >>>>>
> >>
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> >>>>>>         at
> >>>>>
> >> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> >>>>>>         at
> >>>>>
> >>
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> >>>>>
> >>>>>
> >>>>
> >>>
> >>
> >>
> >
>
>

Reply via email to