[
https://issues.apache.org/jira/browse/FLINK-19238?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Yu Li resolved FLINK-19238.
---------------------------
Fix Version/s: 1.12.0
Resolution: Implemented
Merged into master via 93c6256aace5004719241c64d320de5a51f7ec5c
> RocksDB performance issue with low managed memory and high parallelism
> ----------------------------------------------------------------------
>
> Key: FLINK-19238
> URL: https://issues.apache.org/jira/browse/FLINK-19238
> Project: Flink
> Issue Type: Improvement
> Components: Runtime / State Backends
> Affects Versions: 1.11.1
> Reporter: Juha Mynttinen
> Assignee: Juha Mynttinen
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.12.0
>
>
> h2. The issue
> When using {{RocksDBStateBackend}}, it's possible to configure RocksDB so
> that it almost constantly flushes the active memtable, causing high IO and
> CPU usage.
> This happens because this check will be true essentially always
> [https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/include/rocksdb/write_buffer_manager.h#L47].
> h2. Reproducing the issue
> To reproduce the issue, the following needs to happen:
> * Use RocksDB state backend
> * Use managed memory
> * have "low" managed memory size
> * have "high" parallelism (e.g. 5) OR have enough operators (the exact count
> unknown)
> The easiest way to do all this is to do
> {{StreamExecutionEnvironment.createLocalEnvironment}} and creating a simple
> Flink job and setting the parallelism "high enough". Nothing else is needed.
> h2. Background
> Arena memory block size is by default 1/8 of the memtable size
> [https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/db/column_family.cc#L196].
> When the memtable has any data, it'll consume one arena block. The arena
> block size will be higher the "mutable limit". The mutable limit is
> calculated from the shared write buffer manager size. Having low managed
> memory and high parallelism pushes the mutable limit to a too low value.
> h2. Documentation
> In docs
> ([https://ci.apache.org/projects/flink/flink-docs-stable/ops/state/large_state_tuning.html):]
>
>
> "An advanced option (expert mode) to reduce the number of MemTable flushes
> in setups with many states, is to tune RocksDB’s ColumnFamily options (arena
> block size, max background flush threads, etc.) via a RocksDBOptionsFactory".
>
> This snippet in the docs is probably talking about the issue I'm witnessing.
> I think there are two issues here:
>
> 1) it's hard/impossible to know what kind of performance one can expect from
> a Flink application. Thus, it's hard to know if one is suffering from e.g.
> from this performance issue, or if the system is performing normally (and
> inherently being slow).
> 2) even if one suspects a performance issue, it's very hard to find the root
> cause of the performance issue (memtable flush happening frequently). To find
> out this one would need to know what's the normal flush frequency.
>
> Also the doc says "in setups with many states". The same problem is hit when
> using just one state, but "high" parallelism (5).
>
> If the arena block size _ever_ needs to be configured only to "fix" this
> issue, it'd be best if there _never_ was a need to modify arena block size.
> What if we forget even mentioning arena block size in the docs and focus on
> the managed memory size, since managed memory size is something the user does
> tune.
> h1. The proposed fix
> The proposed fix is to log the issue on WARN level and tell the user clearly
> what is happening and how to fix.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)