This graph shows Non-Heap . If the same pattern exists it make sense that
it will try to allocate more memory and then exceed the limit. I can see
the trend for all other containers that has been killed. So my question is
now, what is using non-heap memory?
From
http://mail-archives.apache.org/mod_mbox/flink-user/201707.mbox/%3ccanc1h_u0dqqvbysdaollbemewaxiimtmfjjcribpfpo0idl...@mail.gmail.com%3E
it look like RockDb could be guilty.

I have job using incremental checkpointing and some without, some optimised
for FLASH_SSD. all have same pattern

Lasse



2018-04-10 8:52 GMT+02:00 Lasse Nedergaard <lassenederga...@gmail.com>:

> Hi.
>
> I found the exception attached below, for our simple job. It states that
> our task-manager was killed du to exceed memory limit on 2.7GB. But when I
> look at Flink metricts just 30 sec before it use 1.3 GB heap and 712 MB
> Non-Heap around 2 GB.
> So something else are also using memory inside the conatianer any idea how
> to figure out what?
> As a side note we use RockDBStateBackend with this configuration
>
> env.getCheckpointConfig().setMinPauseBetweenCheckpoints((long)(config.checkPointInterval
>  * 0.75));
> env.enableCheckpointing(config.checkPointInterval, 
> CheckpointingMode.AT_LEAST_ONCE);
> env.setStateBackend(new RocksDBStateBackend(config.checkpointDataUri));
>
> Where checkpointDataUri point to S3
>
> Lasse Nedergaard
>
> 2018-04-09 16:52:01,239 INFO  org.apache.flink.yarn.
> YarnFlinkResourceManager                - Diagnostics for container
> container_1522921976871_0001_01_000079 in state COMPLETE :
> exitStatus=Pmem limit exceeded (-104) diagnostics=Container
> [pid=30118,containerID=container_1522921976871_0001_01_000079] is running
> beyond physical memory limits. Current usage: 2.7 GB of 2.7 GB physical
> memory used; 4.9 GB of 13.4 GB virtual memory used. Killing container.
>
> Dump of the process-tree for container_1522921976871_0001_01_000079 :
>
> |- PID PPID PGRPID SESSID CMD_NAME USER_MODE_TIME(MILLIS)
> SYSTEM_TIME(MILLIS) VMEM_USAGE(BYTES) RSSMEM_USAGE(PAGES) FULL_CMD_LINE
>
> |- 30136 30118 30118 30118 (java) 245173 68463 5193723904 703845
> /usr/lib/jvm/java-openjdk/bin/java -Xms2063m -Xmx2063m
> -Dlog.file=/var/log/hadoop-yarn/containers/application_
> 1522921976871_0001/container_1522921976871_0001_01_000079/taskmanager.log
> -Dlogback.configurationFile=file:./logback.xml
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskManager
> --configDir .
>
> |- 30118 30116 30118 30118 (bash) 0 0 115818496 674 /bin/bash -c
> /usr/lib/jvm/java-openjdk/bin/java -Xms2063m -Xmx2063m
> -Dlog.file=/var/log/hadoop-yarn/containers/application_
> 1522921976871_0001/container_1522921976871_0001_01_000079/taskmanager.log
> -Dlogback.configurationFile=file:./logback.xml
> -Dlog4j.configuration=file:./log4j.properties 
> org.apache.flink.yarn.YarnTaskManager
> --configDir . 1> /var/log/hadoop-yarn/containers/application_
> 1522921976871_0001/container_1522921976871_0001_01_000079/taskmanager.out
> 2> /var/log/hadoop-yarn/containers/application_
> 1522921976871_0001/container_1522921976871_0001_01_000079/taskmanager.err
>
>
> 2018-04-09 16:51:26,659 DEBUG org.trackunit.tm2.LogReporter
>                   - gauge.ip-10-1-1-181.taskmanager.container_
> 1522921976871_0001_01_000079.Status.JVM.Memory.Heap.Used=1398739496
>
>
> 2018-04-09 16:51:26,659 DEBUG org.trackunit.tm2.LogReporter
>                   - gauge.ip-10-1-1-181.taskmanager.container_
> 1522921976871_0001_01_000079.Status.JVM.Memory.NonHeap.Used=746869520
>
>
>
>
>
> 2018-04-09 23:52 GMT+02:00 Ken Krugler <kkrugler_li...@transpac.com>:
>
>> Hi Chesnay,
>>
>> Don’t know if this helps, but I’d run into this as well, though I haven’t
>> hooked up YourKit to analyze exactly what’s causing the memory problem.
>>
>> E.g. after about 3.5 hours running locally, it failed with memory issues.
>>
>> In the TaskManager logs, I start seeing exceptions in my code….
>>
>> java.lang.OutOfMemoryError: GC overhead limit exceeded
>>
>> And then eventually...
>>
>> 2018-04-07 21:55:25,686 WARN  
>> org.apache.flink.runtime.accumulators.AccumulatorRegistry
>>     - Failed to serialize accumulators for task.
>> java.lang.OutOfMemoryError: GC overhead limit exceeded
>>
>> Immediately after this, one of my custom functions gets a close() call,
>> and I see a log msg about it "switched from RUNNING to FAILED”.
>>
>> After this, I see messages that the job is being restarted, but the
>> TaskManager log output abruptly ends.
>>
>> In the Job Manager log, this is what is output following the time of the
>> last TaskManager logging output:
>>
>> 2018-04-07 21:57:33,702 INFO  
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator
>>     - Triggering checkpoint 129 @ 1523163453702
>> 2018-04-07 21:58:43,916 WARN  akka.remote.ReliableDeliverySupervisor
>>                    - Association with remote system [
>> akka.tcp://fl...@kens-mbp.hsd1.ca.comcast.net:63780] has failed, address
>> is now gated for [5000] ms. Reason: [Disassociated]
>> 2018-04-07 21:58:51,084 WARN  akka.remote.transport.netty.NettyTransport
>>                    - Remote connection to [null] failed with
>> java.net.ConnectException: Connection refused: kens-
>> mbp.hsd1.ca.comcast.net/192.168.3.177:63780
>> 2018-04-07 21:58:51,086 WARN  akka.remote.ReliableDeliverySupervisor
>>                    - Association with remote system [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780] has failed, address is now
>> gated for [5000] ms. Reason: [Association failed with [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780]] Caused by: [Connection
>> refused: kens-mbp.hsd1.ca.comcast.net/192.168.3.177:63780]
>> 2018-04-07 21:59:01,047 WARN  akka.remote.transport.netty.NettyTransport
>>                    - Remote connection to [null] failed with
>> java.net.ConnectException: Connection refused:
>> kens-mbp.hsd1.ca.comcast.net/192.168.3.177:63780
>> 2018-04-07 21:59:01,050 WARN  akka.remote.ReliableDeliverySupervisor
>>                    - Association with remote system [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780] has failed, address is now
>> gated for [5000] ms. Reason: [Association failed with [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780]] Caused by: [Connection
>> refused: kens-mbp.hsd1.ca.comcast.net/192.168.3.177:63780]
>> 2018-04-07 21:59:11,057 WARN  akka.remote.ReliableDeliverySupervisor
>>                    - Association with remote system [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780] has failed, address is now
>> gated for [5000] ms. Reason: [Association failed with [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780]] Caused by: [Connection
>> refused: kens-mbp.hsd1.ca.comcast.net/192.168.3.177:63780]
>> 2018-04-07 21:59:11,058 WARN  akka.remote.transport.netty.NettyTransport
>>                    - Remote connection to [null] failed with
>> java.net.ConnectException: Connection refused:
>> kens-mbp.hsd1.ca.comcast.net/192.168.3.177:63780
>> 2018-04-07 21:59:21,049 WARN  akka.remote.transport.netty.NettyTransport
>>                    - Remote connection to [null] failed with
>> java.net.ConnectException: Connection refused:
>> kens-mbp.hsd1.ca.comcast.net/192.168.3.177:63780
>> 2018-04-07 21:59:21,049 WARN  akka.remote.ReliableDeliverySupervisor
>>                    - Association with remote system [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780] has failed, address is now
>> gated for [5000] ms. Reason: [Association failed with [akka.tcp://
>> fl...@kens-mbp.hsd1.ca.comcast.net:63780]] Caused by: [Connection
>> refused: kens-mbp.hsd1.ca.comcast.net/192.168.3.177:63780]
>> 2018-04-07 21:59:21,056 WARN  akka.remote.RemoteWatcher
>>                   - Detected unreachable: [akka.tcp://fl...@kens-mbp.hsd
>> 1.ca.comcast.net:63780]
>> 2018-04-07 21:59:21,063 INFO  org.apache.flink.runtime.jobmanager.JobManager
>>                - Task manager akka.tcp://flink@kens-mbp.hsd1
>> .ca.comcast.net:63780/user/taskmanager terminated.
>> 2018-04-07 21:59:21,064 INFO  
>> org.apache.flink.runtime.executiongraph.ExecutionGraph
>>        - FetchUrlsFunction for sitemap -> ParseSiteMapFunction ->
>> OutlinkToStateUrlFunction (1/1) (3e9374d1bf5fdb359e3a624a4d5d659b)
>> switched from RUNNING to FAILED.
>> java.lang.Exception: TaskManager was lost/killed:
>> c51d3879b6244828eb9fc78c943007ad @ kens-mbp.hsd1.ca.comcast.net
>> (dataPort=63782)
>>
>> — Ken
>>
>>
>>
>> On Apr 9, 2018, at 12:48 PM, Chesnay Schepler <ches...@apache.org> wrote:
>>
>> We will need more information to offer any solution. The exception simply
>> means that a TaskManager shut down, for which there are a myriad of
>> possible explanations.
>>
>> Please have a look at the TaskManager logs, they may contain a hint as to
>> why it shut down.
>>
>> On 09.04.2018 16:01, Javier Lopez wrote:
>>
>> Hi,
>>
>> "are you moving the job  jar to  the ~/flink-1.4.2/lib path ?  " -> Yes,
>> to every node in the cluster.
>>
>> On 9 April 2018 at 15:37, miki haiat <miko5...@gmail.com> wrote:
>>
>>> Javier
>>> "adding the jar file to the /lib path of every task manager"
>>> are you moving the job  jar to  the* ~/flink-1.4.2/lib path* ?
>>>
>>> On Mon, Apr 9, 2018 at 12:23 PM, Javier Lopez <javier.lo...@zalando.de>
>>> wrote:
>>>
>>>> Hi,
>>>>
>>>> We had the same metaspace problem, it was solved by adding the jar file
>>>> to the /lib path of every task manager, as explained here
>>>> https://ci.apache.org/projects/flink/flink-docs-release
>>>> -1.4/monitoring/debugging_classloading.html#avoiding-dynamic
>>>> -classloading. As well we added these java options:
>>>> "-XX:CompressedClassSpaceSize=100M -XX:MaxMetaspaceSize=300M
>>>> -XX:MetaspaceSize=200M "
>>>>
>>>> From time to time we have the same problem with TaskManagers
>>>> disconnecting, but the logs are not useful. We are using 1.3.2.
>>>>
>>>> On 9 April 2018 at 10:41, Alexander Smirnov <
>>>> alexander.smirn...@gmail.com> wrote:
>>>>
>>>>> I've seen similar problem, but it was not a heap size, but Metaspace.
>>>>> It was caused by a job restarting in a loop. Looks like for each
>>>>> restart, Flink loads new instance of classes and very soon in runs out of
>>>>> metaspace.
>>>>>
>>>>> I've created a JIRA issue for this problem, but got no response from
>>>>> the development team on it: https://issues.apache.org/
>>>>> jira/browse/FLINK-9132
>>>>>
>>>>>
>>>>> On Mon, Apr 9, 2018 at 11:36 AM 王凯 <wangka...@163.com> wrote:
>>>>>
>>>>>> thanks a lot,i will try it
>>>>>>
>>>>>> 在 2018-04-09 00:06:02,"TechnoMage" <mla...@technomage.com> 写道:
>>>>>>
>>>>>> I have seen this when my task manager ran out of RAM.  Increase the
>>>>>> heap size.
>>>>>>
>>>>>> flink-conf.yaml:
>>>>>> taskmanager.heap.mb
>>>>>> jobmanager.heap.mb
>>>>>>
>>>>>> Michael
>>>>>>
>>>>>> On Apr 8, 2018, at 2:36 AM, 王凯 <wangka...@163.com> wrote:
>>>>>>
>>>>>> <QQ图片20180408163927.png>
>>>>>> hi all, recently, i found a problem,it runs well when start. But
>>>>>> after long run,the exception display as above,how can resolve it?
>>>>>>
>>>>>>
>> --------------------------------------------
>> http://about.me/kkrugler
>> +1 530-210-6378
>>
>>
>

Reply via email to