[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16902614#comment-16902614
 ] 

Akshay Iyangar commented on FLINK-7289:
---------------------------------------

[~mikekap] - We too have the same issue when running Flink in Kubernetes. We 
tried to use the fix that you have to restrict the WriteBufferManger size but 
even then we see the memory for rocks continously increasing. 
{code:java}
Options.write_buffer_size: 67108864
Options.max_write_buffer_number: 2
Options.max_open_files: -1
{code}
We set the write buffer manager size to 8gb. 

We use 10Gb as the heap size per TM . 

Each node/TM we use is a 32Gb box. but as time proceeds our memory keeps 
increasing. Our JVM heap memory is pretty constant, we used a profiler to 
verify this, but for rocks i'm not sure what is the best way to find what is 
consuming all the memory.

I would appreciate if you can push me in the right direction to see what more 
can be tuned wrt rocks.

Attaching the complete conf for rocks

[^completeRocksdbConfig.txt]

 

 

> Memory allocation of RocksDB can be problematic in container environments
> -------------------------------------------------------------------------
>
>                 Key: FLINK-7289
>                 URL: https://issues.apache.org/jira/browse/FLINK-7289
>             Project: Flink
>          Issue Type: Improvement
>          Components: Runtime / State Backends
>    Affects Versions: 1.2.0, 1.3.0, 1.4.0
>            Reporter: Stefan Richter
>            Priority: Major
>         Attachments: completeRocksdbConfig.txt
>
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to