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

Yu Li commented on FLINK-7289:
------------------------------

Thanks for the note [~srichter], I agree this is a great suggestion (thanks 
[~mikekap]).

Reading the document we could see after 5.6 the newly introduced write buffer 
manager could limit the total memory size for RocksDB. I think we should supply 
a way to adopt the usage of write buffer manager in Flink as Mike suggested.

On the other hand, I have some concern about the possible downside of this 
"cost memory used in memtable to block cache" mechanism, that will it cause 
performance downgrade for read heavy workload? If so, what extent that would 
be? I believe with such information we could further describe pros and cons and 
make it easier for user to decide how to set the db options. So please give me 
more time to check this out and will be back right after I got some answer. 
Please also let me know if you have some study on this part [~mikekap]. Thanks.

> 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
>
> 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.3#76005)

Reply via email to