flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Stefan Richter <s.rich...@data-artisans.com>
Subject Re: Possible JVM native memory leak
Date Wed, 18 Jan 2017 10:05:50 GMT
Hi,

The answer to question one is clearly yes, and you can configure RocksDB through the DBOptions.

Question two is obviously more tricky with the given information. But it is surely possible
that some resources are not properly released. All classes from the RocksDB Java API have
a safety net that releases resources in their finalizer, but given your huge heap size and
the small size (on heap) of those objects, they simply might not become garbage collected
until you run into the problem. You could take a heap dump and search for unreferenced instances
of RocksDB objects that have not yet been closed/diposed.

For curiosity, I also wonder why you have so many Threads, is this justified by the jobs you
are running? And why do you chose 40GB of heap? When using RocksDB state backend, your heavy
state should typically be in RocksDB and not on the heap, so you better configure a smaller
heap and larger native memory.

We might find some more time to look into this after the Flink 1.2 release, but this problem
could also be solved in the new version where many parts of the RocksDB backend have been
rewritten.

Best,
Stefan 

> Am 17.01.2017 um 14:52 schrieb Avihai Berkovitz <Avihai.Berkovitz@microsoft.com>:
> 
> Hello,
>  
> I am running a streaming job on a small cluster, and after a few hours I noticed that
my TaskManager processes are being killed by the OOM killer. The processes were using too
much memory. After a bit of monitoring, I have the following status:
> The maximum heap size (Xmx) is 40000M
> Native Memory Tracking reports that the process has 44180459KB committed, which is reasonable
given the GC and threads overhead (the full summery report is attached later)
> There are 644 threads
> The Status.JVM.Memory.NonHeap.Committed metric is 245563392
> The Status.JVM.Memory.Direct.MemoryUsed metric is 354777032
> Using pmap we find that the private committed memory is 54879428K and mapped is 62237852K
>  
> So we have about 10GB of memory that was allocated in the process but is unknown to the
JVM itself.
>  
> Some more info:
> I am running Flink 1.1.4
> I am using RocksDB for state
> The state is saved to Azure Blob Store, using the NativeAzureFileSystem HDFS connector
over the wasbs protocol
> The cluster is a standalone HA cluster
> The machine is an Ubuntu 14.04.5 LTS 64 bit server
> I have around 2 GB of state per TaskManager
>  
> Another thing I noticed is that the job sometimes fails (due to external DB connectivity
issues) and is restarted automatically as expected. But in some cases the failures also cause
one or more of the following error logs:
> Could not close the file system output stream. Trying to delete the underlying file.
> Could not discard the 1th operator state of checkpoint 93 for operator Operator2.
> java.lang.NullPointerException: null
>                 at org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointThread.run(StreamTask.java:953)
~[flink-dist_2.10-1.1.4.jar:1.1.4]
>  
> I have 2 theories, and I hope to hear any ideas from you:
> RocksDB uses this memory for internal caching. If so, how can this usage be tracked,
and what options can tune and limit it?
> Internal RocksDB objects are not being disposed of properly, probably during the aforementioned
job restarts, and so we have a growing memory leak. If so, do you have any idea what can cause
this?
>  
> Thank you,
> Avihai
>  
>  
> Attached Native Memory Tracking (jcmd <PID> VM.native_memory summary):
>  
> Total: reserved=44399603KB, committed=44180459KB
> -                 Java Heap (reserved=40960000KB, committed=40960000KB)
>                             (mmap: reserved=40960000KB, committed=40960000KB)
>  
> -                     Class (reserved=134031KB, committed=132751KB)
>                             (classes #22310)
>                             (malloc=2959KB #43612)
>                             (mmap: reserved=131072KB, committed=129792KB)
>  
> -                    Thread (reserved=716331KB, committed=716331KB)
>                             (thread #694)
>                             (stack: reserved=712404KB, committed=712404KB)
>                             (malloc=2283KB #3483)
>                             (arena=1644KB #1387)
>  
> -                      Code (reserved=273273KB, committed=135409KB)
>                             (malloc=23673KB #30410)
>                             (mmap: reserved=249600KB, committed=111736KB)
>  
> -                        GC (reserved=1635902KB, committed=1635902KB)
>                             (malloc=83134KB #70605)
>                             (mmap: reserved=1552768KB, committed=1552768KB)
>  
> -                  Compiler (reserved=1634KB, committed=1634KB)
>                             (malloc=1504KB #2062)
>                             (arena=131KB #3)
>  
> -                  Internal (reserved=575283KB, committed=575283KB)
>                             (malloc=575251KB #106644)
>                             (mmap: reserved=32KB, committed=32KB)
>  
> -                    Symbol (reserved=16394KB, committed=16394KB)
>                             (malloc=14468KB #132075)
>                             (arena=1926KB #1)
>  
> -    Native Memory Tracking (reserved=6516KB, committed=6516KB)
>                             (malloc=338KB #5024)
>                             (tracking overhead=6178KB)
>  
> -               Arena Chunk (reserved=237KB, committed=237KB)
>                             (malloc=237KB)
>  
> -                   Unknown (reserved=80000KB, committed=0KB)
>                             (mmap: reserved=80000KB, committed=0KB)


Mime
View raw message