Hi Steve,
RocksDB does not contribute to the JVM direct memory. RocksDB off-heap
memory consumption is part of managed memory [1].
You got `OutOfMemoryError: Direct buffer memory` which is related to
the JVM direct memory, also off-heap but managed by JVM.
The JVM direct memory limit depends on
We recently migrated to Flink 1.10, but are experiencing some issues with
memory.
Our cluster is:
1) Running inside of Kubernetes
2) Running in HA mode
3) Checkpointing/Savepointing to an HDFS cluster inside of Kubernetes
4) Using RocksDB for checkpointing
5) Running on m5d.4xlarge EC2 instances
Hi,
OOMs from metaspace probably mean that your jars are not releasing some
resources:
https://ci.apache.org/projects/flink/flink-docs-release-1.3/monitoring/debugging_classloading.html#unloading-of-dynamically-loaded-classes
Hi Piotr,
We found out which one was the problem in the workers. After setting a
value for XX:MaxMetaspaceSize we started to get OOM exceptions from the
metaspace. We found out how Flink manages the User classes here
Hi,
I have been able to observe some off heap memory “issues” by submitting Kafka
job provided by Javier Lopez (in different mailing thread).
TL;DR;
There was no memory leak, just memory pool “Metaspace” and “Compressed Class
Space” are growing in size over time and are only rarely garbage
Unfortunately the issue I've opened [1] was not a problem of Flink but was
just caused by an ever increasing job plan.
So no help from that..Let's hope to find out the real source of the problem.
Maybe using -Djdk.nio.maxCachedBufferSize could help (but I didn't try it
yet)
Best,
Flavio
[1]
Hi,
We saw a similar issue in one of our job due to ByteBuffer memory leak[1].
We fixed it using the solution in the article, setting
-Djdk.nio.maxCachedBufferSize
This variable is available for Java > 8u102
Best regards,
Kien
[1]http://www.evanjones.ca/java-bytebuffer-leak.html
On
We also faced the same problem, but the number of jobs we can run before
restarting the cluster depends on the volume of the data to shuffle around
the network. We even had problems with a single job and in order to avoid
OOM issues we had to put some configuration to limit Netty memory usage,
Hi Robert,
Sorry to reply this late. We did a lot of tests, trying to identify if the
problem was in our custom sources/sinks. We figured out that none of our
custom components is causing this problem. We came up with a small test,
and realized that the Flink nodes run out of non-heap JVM memory
Thanks for letting us know!
> On 18. Sep 2017, at 11:36, PedroMrChaves wrote:
>
> Hello,
>
> Sorry for the delay.
>
> The buffer memory of the Kafka consumer was piling up. Once I updated to the
> 1.3.2 version the problem no longer occurred.
>
> Pedro.
>
>
>
>
Hello,
Sorry for the delay.
The buffer memory of the Kafka consumer was piling up. Once I updated to the
1.3.2 version the problem no longer occurred.
Pedro.
-
Best Regards,
Pedro Chaves
--
Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
Hi Javier,
I'm not aware of such issues with Flink, but if you could give us some more
details on your setup, I might get some more ideas on what to look for.
are you using the RocksDBStateBackend? (RocksDB is doing some JNI
allocations, that could potentially leak memory)
Also, are you passing
Hi all,
we are starting a lot of Flink jobs (streaming), and after we have started
200 or more jobs we see that the non-heap memory in the taskmanagers
increases a lot, to the point of killing the instances. We found out that
every time we start a new job, the committed non-heap memory increases
Thanks Stephan, any pointers on how managed memory is used in streaming
application will really help.
Regards,
Govind
> On Aug 24, 2017, at 1:53 AM, Stephan Ewen wrote:
>
> Hi!
>
> RocksDB will be used when it is selected as the state backend, independent of
> the
Hi!
RocksDB will be used when it is selected as the state backend, independent
of the checkpointing configuration.
Using RocksDB as the state backend, Flink will have some objects on the
heap, like timers (we will move them to RocksDB as well in the near future)
but the majority will be off
I have couple more questions regarding flink's jvm memory.
In a streaming application what is managed memory used for? I read from a
blog that all objects created inside the user function will go into
unmanaged memory. Where does the managed key/ operator state state reside?
Also when does the
One would need to look at your code and possible on some heap statistics. Maybe
something wrong happens when you cache them (do you use a 3rd party library or
your own implementation?). Do you use a stable version of your protobuf library
(not necessarily the most recent). You also may want to
Hi,
I have a pipeline running on flink which ingests around 6k messages per
second. Each message is around 1kb and it passes through various stages
like filter, 5 sec tumbling window per key etc.. and finally flatmap to
computation before sending it to kafka sink. The data is first ingested as
y allocated memory.
>>
>> Regards,
>> Pedro Chaves.
>>
>>
>>
>> -
>> Best Regards,
>> Pedro Chaves
>> --
>> View this message in context: http://apache-flink-user-maili
>> ng-list-archive.2336050.n4.nabble.com/FlinkKafkaConsumer0
>> 10-Memory-Issue-tp14342p14445.html
>> Sent from the Apache Flink User Mailing List archive. mailing list
>> archive at Nabble.com.
>>
>
>
://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/FlinkKafkaConsumer010-Memory-Issue-tp14342p14445.html
Sent from the Apache Flink User Mailing List archive. mailing list archive at
Nabble.com.
Hi,
I couldn’t seem to reproduce this.
Taking another look at your description, one thing I spotted was that your
Kafka broker installation versions are 0.10.1.0, while the Kafka consumer uses
Kafka clients of version 0.10.0.1 (by default, as shown in your logs).
I’m wondering whether or not
this message in context:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/FlinkKafkaConsumer010-Memory-Issue-tp14342p14445.html
Sent from the Apache Flink User Mailing List archive. mailing list archive at
Nabble.com.
ache-flink-user-mailing-list-archive.2336050.n4.nabble.com/file/n14342/taskmanager.log>
-
Best Regards,
Pedro Chaves
--
View this message in context:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/FlinkKafkaConsumer010-Memory-Issue-tp14342.html
Sent from the Apache Flink User Mailing List archive. mailing list archive at
Nabble.com.
t;
-
Best Regards,
Pedro Chaves
--
View this message in context:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/FlinkKafkaConsumer010-Memory-Issue-tp14342.html
Sent from the Apache Flink User Mailing List archive. mailing list archive at
Nabble.com.
t; 15121
> Swap: 0 0 0/
>
> after having the job running for about 5 min:
> free -m
> / totalusedfree shared buff/cache
> available
> Mem: 1581798195150 24 847
> 5547
> Swap:
0 24 847
5547
Swap: 0 0 0
/
taskmanager.log
<http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/file/n14342/taskmanager.log>
-
Best Regards,
Pedro Chaves
--
View this message in context:
http://apache-flink
ew this message in context:
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/yarn-kill-container-due-to-running-beyond-physical-memory-limits-How-can-i-debug-memory-issue-tp7296p7325.html
> Sent from the Apache Flink User Mailing List archive. mailing list archive
Size=4608m
Regards
Prateek
-
--
View this message in context:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/yarn-kill-container-due-to-running-beyond-physical-memory-limits-How-can-i-debug-memory-issue-tp7296p7325.html
Sent from the Apache Flink User Mailing List archive.
if i tried to run my application using below configuration then also
> facing same issue.
>
>flink run -m yarn-cluster -yn 15 -ytm 4096 -ys 1
>
>
>
> Regards
> Prateek
>
>
>
>
>
> --
> View this message in context:
> http://apach
.
flink run -m yarn-cluster -yn 15 -ytm 4096 -ys 1
Regards
Prateek
--
View this message in context:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/yarn-kill-container-due-to-running-beyond-physical-memory-limits-How-can-i-debug-memory-issue-tp7296p7317.html
prateekarora <prateek.arora...@gmail.com>:
> Hi
>
> I am running flink 1.0.2 with Yarn .
>
> After running application for some time , Yarn kill my container due to
> running beyond physical memory limits .
>
> how can i debug memory issue ?
>
>
Hi
I am running flink 1.0.2 with Yarn .
After running application for some time , Yarn kill my container due to
running beyond physical memory limits .
how can i debug memory issue ?
below are the logs :
Container container_1463184272818_0165_01_12 is completed with
diagnostics: Container
32 matches
Mail list logo