You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Sihua Zhou (JIRA)" <ji...@apache.org> on 2018/04/20 05:49:00 UTC

[jira] [Commented] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

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

Sihua Zhou commented on FLINK-7289:
-----------------------------------

Hi [~srichter], I think it's quite hard to tell what is the best configuration for RocksDB, but there is one experience we had when using the RocksDBBackend, that is when we using RocksDBBackend we always set the {{taskmanager.memory.size}}, this parameter define the OFF-HEAP memory size that used by TM, it maybe a bit weird that we set this parameter for a stream job (because AFAIK currently only the batch job will use the OFF-HEAP memory to create direct memory buffer), the purpose that we set this parameter is we want to reserve this OFF-HEAP memory for RocksDB, even thought RocksDB doesn't regard it or controlled by it, but it split the total memory apart, one is HEAP memory used by JVM and controlled quite well with GC, other is OFF-HEAP that we reserve for RocksDB. This help us to be safer, but the parameter's name(taskmanager.memory.size) is quite weird regard of it's purpose in a stream job at a first glance. What do you think?

> 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: State Backends, Checkpointing
>    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)