Hi William

Have you ever set the memory limit of your taskmanager pod when launching it in 
k8s? If not, I'm afraid your node might come across node out-of-memory [1]. You 
could increase the limit by analyzing your memory usage
When talking about the memory usage of RocksDB, a rough calculation formula 
could be: block-cache-memory + column-family-number * write-buffer-memory * 
write-buffer-number + index&filter memory. The block cache, write buffer 
memory&number could be mainly configured. And the column-family number is 
decided by the state number within your operator. The last part of index&filter 
memory cannot be measured well only if you also cache them in block cache [2] 
(but this would impact the performance).
If you want to the memory stats of rocksDB, turn on the native metrics of 
RocksDB [3] is a good choice.


[1] 
https://kubernetes.io/docs/tasks/administer-cluster/out-of-resource/#node-oom-behavior
[2] 
https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB#indexes-and-filter-blocks
[3] 
https://ci.apache.org/projects/flink/flink-docs-release-1.8/ops/config.html#rocksdb-native-metrics

Best
Yun Tang
________________________________
From: Xintong Song <tonysong...@gmail.com>
Sent: Wednesday, July 24, 2019 11:59
To: wvl <lee...@gmail.com>
Cc: user <user@flink.apache.org>
Subject: Re: Memory constrains running Flink on Kubernetes

Hi,

Flink acquires these 'Status_JVM_Memory' metrics through the MXBean library. 
According to MXBean document, non-heap is "the Java virtual machine manages 
memory other than the heap (referred as non-heap memory)". Not sure whether 
that is equivalent to the metaspace. If the '-XX:MaxMetaspaceSize', it should 
trigger metaspcae clean up when the limit is reached.

As for RocksDB, it mainly uses non-java memory. Heap, non-heap and direct 
memory could be considered as java memory (or at least allocated through the 
java process). That means, RocksDB is actually using the memory that is 
accounted in the total K8s container memory but not accounted in neither of 
java heap / non-heap / direct memory, which in your case the 1GB unaccounted. 
To leave more memory for RocksDB, you need to either configure more memory for 
the K8s containers, or configure less java memory through the config option 
'taskmanager.heap.size'.

The config option 'taskmanager.heap.size', despite the 'heap' in its key, also 
accounts for network memory (which uses direct buffers). Currently, memory 
configurations in Flink is quite complicated and confusing. The community is 
aware of this, and is planing for an overall improvement.

To my understanding, once you set '-XX:MaxMetaspaceSize', there should be 
limits on heap, non-heap and direct memory in JVM. You should be able to find 
which part that requires memory more than the limit from the java OOM error 
message. If there is no java OOM but a K8s container OOM, then it should be 
non-java memory used by RocksDB.

[1] 
https://docs.oracle.com/javase/8/docs/api/java/lang/management/MemoryMXBean.html


Thank you~

Xintong Song


On Tue, Jul 23, 2019 at 8:42 PM wvl <lee...@gmail.com<mailto:lee...@gmail.com>> 
wrote:
Hi,

We're running a relatively simply Flink application that uses a bunch of state 
in RocksDB on Kubernetes.
During the course of development and going to production, we found that we were 
often running into memory issues made apparent by Kubernetes OOMKilled and Java 
OOM log events.

In order to tackle these, we're trying to account for all the memory used in 
the container, to allow proper tuning.
Metric-wise we have:
- container_memory_working_set_bytes = 6,5GB
- flink_taskmanager_Status_JVM_Memory_Heap_Max =  4,7GB
- flink_taskmanager_Status_JVM_Memory_NonHeap_Used = 325MB
- flink_taskmanager_Status_JVM_Memory_Direct_MemoryUsed = 500MB

This is my understanding based on all the documentation and observations:
container_memory_working_set_bytes will be the total amount of memory in use, 
disregarding OS page & block cache.
Heap will be heap.
NonHeap is mostly the metaspace.
Direct_Memory is mostly network buffers.

Running the numbers I have 1 GB unaccounted for. I'm also uncertain as to 
RocksDB. According to the docs RocksDB has a "Column Family Write Buffer" where 
"You need to budget for 2 x your worst case memory use".
We have 17 ValueStateDescriptors (ignoring state for windows) which I'm 
assuming corresponds to a "Column Family" in RockDB. Meaning our budget should 
be around 2GB.
Is this accounted for in one of the flink_taskmanager metrics above? We've also 
enabled various rocksdb metrics, but it's unclear where this Write Buffer 
memory would be represented.

Finally, we've seen that when our job has issues and is restarted rapidly, 
NonHeap_Used grows from an initial 50Mb to 700MB, before our containers are 
killed. We're assuming this is due
to no form of cleanup in the metaspace as classes get (re)loaded.

These are our taskmanager JVM settings: -XX:+UseG1GC -XX:MaxDirectMemorySize=1G 
-XX:+UnlockExperimentalVMOptions -XX:+UseCGroupMemoryLimitForHeap 
-XX:MaxRAMFraction=2
With flink config:
      taskmanager.heap.size: 5000m
      state.backend: rocksdb
      state.backend.incremental: true
      state.backend.rocksdb.timer-service.factory: ROCKSDB

Based on what we've observed we're thinking about setting -XX:MaxMetaspaceSize 
to a reasonable value, so that we at least get an error message which can 
easily be traced back to the behavior we're seeing.

Okay, all that said let's sum up what we're asking here:
- Is there any more insight into how memory is accounted for than our current 
metrics?
- Which metric, if any accounts for RocksDB memory usage?
- What's going on with the Metaspace growth we're seeing during job restarts, 
is there something we can do about this such as setting -XX:MaxMetaspaceSize?
- Any other tips to improve reliability running in resource constrained 
environments such as Kubernetes?

Thanks,

William

Reply via email to