[
https://issues.apache.org/jira/browse/FLINK-15368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17010018#comment-17010018
]
Yun Tang commented on FLINK-15368:
----------------------------------
{quote}When is memory reserved? Is that every 1MB when a "dummy block" is
inserted into the cache to represent the used write buffer memory?
{quote}
The memory allocator in RocksDB is called {{arena}}, and {{memtable}} use a
{{ConcurrentArena}}to allocate memory. When {{memtable}} is initialized, the
{{arena_block_size}} (which is 1/8 of write buffer by default = 8MB) would be
used [as the block
size|https://github.com/dataArtisans/frocksdb/blob/c724d41fab7f9f09f9676dfccc6d210a191da4d6/db/memtable.cc#L71].
When an entry entered into this memtable, write buffer manager would reserve a
{{kBlockSize}} block of memory (there is only one exception if we allocate a
block larger than {{kBlockSize/4}}, we would direct allocate that amount memory
instead of a full block)
The picture below describes the blocks in {{arena}}:
!rocksdb-arena-block.png|width=743,height=221!
To improve the performance, rocksdb would use cache shards to reserve the
memory. In other words, the 8MB block would be split into 8 * 1MB dummy blocks.
As you can see, these are actually reserved memory instead of used memory.
{quote}the write buffer ration: Do you have any insights from running this
during tests which value is on average more beneficial for performance?
{quote}
I think we could use flink state benchmark to test write buffer manager with
actual no multi states and no multi RocksDB instances to see what write buffer
ratio could behaves closely near original default behavior. This need some time
and experiments to get the result. However, from what I know, some guys really
depends on predefined {{PredefinedOptions.SPINNING_DISK_OPTIMIZED_HIGH_MEM}}
which changes the ratio of write buffer vs block cache. Even we could behaves
close to original default parameters, I'm afraid that would also impact the
behaviors if user already adopt to use
{{PredefinedOptions.SPINNING_DISK_OPTIMIZED_HIGH_MEM}}.
> Add end-to-end test for controlling RocksDB memory usage
> --------------------------------------------------------
>
> Key: FLINK-15368
> URL: https://issues.apache.org/jira/browse/FLINK-15368
> Project: Flink
> Issue Type: Sub-task
> Components: Runtime / State Backends
> Affects Versions: 1.10.0
> Reporter: Yu Li
> Assignee: Yun Tang
> Priority: Critical
> Fix For: 1.10.0
>
> Attachments: rocksdb-arena-block.png
>
>
> We need to add an end-to-end test to make sure the RocksDB memory usage
> control works well, especially under the slot sharing case.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)