You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Yu Li (Jira)" <ji...@apache.org> on 2020/01/10 16:27:00 UTC
[jira] [Comment Edited] (FLINK-15368) Add end-to-end test for
controlling RocksDB memory usage
[ https://issues.apache.org/jira/browse/FLINK-15368?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17007221#comment-17007221 ]
Yu Li edited comment on FLINK-15368 at 1/10/20 4:26 PM:
--------------------------------------------------------
Thanks for the efforts and update [~yunta]! Below are some comments and suggestions according to the testing output and investigation result:
# We should write explicitly in our document not to set {{optimizeForPointLookup}} when enabling RocksDB memory control.
# We cannot rely on {{strict_capacity_limit}} until [RocksDB#6247|https://github.com/facebook/rocksdb/issues/6247] is resolved. In another word, since the issue on {{strict_capacity_limit}} could hardly be resolved soon, we need to work out a work-around solution for 1.10 release.
# From the [RocksDB document of WriteBufferManager|https://github.com/facebook/rocksdb/wiki/Write-Buffer-Manager] it should be able to limit the total memory of memtables. More specific, the document says "a flush will be triggered if total mutable memtable size exceeds 90% of the limit". However, from the [implementation|https://github.com/facebook/rocksdb/blob/e8263dbdaad0546c54bddd01a8454c2e750a86c2/include/rocksdb/write_buffer_manager.h#L55] we could tell it's not following the document (write buffer could go up to 150% of the size), which is really a surprise.
# Based on the fact of #2 and #3, I suggest we work around by doing an internal computation and setting the size of {{Cache}} and {{WriteBufferManager}} accordingly: assume the memory limit is {{N}} and the write ratio is R, we have {{1.5*size_of_write_buffer_manager=R*N}}, {{1.5*size_of_write_buffer_manager+size_of_others=N}}, and {{size_of_write_buffer_manager+size_of_others=size_of_cache}}, then we know {{size_of_write_buffer_manager=2*N*R/3}} and {{size_of_cache=N*(3-R)/3}}
# For the additional cost of pinned iterator and/or index, it should be relatively small, and if it indeed cause out-of-memory, we suggest to use {{taskmanager.memory.task.offheap}} to cover this part.
# Note that we will be able to get rid of the kind-of-complicated work-around right after the rocksdb bug is fixed.
What do you think?
[~sewen] please also shed some lights here, thanks.
was (Author: carp84):
Thanks for the efforts and update [~yunta]! Below are some comments and suggestions according to the testing output and investigation result:
# We should write explicitly in our document not to set {{optimizeForPointLookup}} when enabling RocksDB memory control.
# We cannot rely on {{strict_capacity_limit}} until [RocksDB#6247|https://github.com/facebook/rocksdb/issues/6247] is resolved. In another word, since the issue on {{strict_capacity_limit}} could hardly be resolved soon, we need to work out a work-around solution for 1.10 release.
# From the [RocksDB document of WriteBufferManager|https://github.com/facebook/rocksdb/wiki/Write-Buffer-Manager] it should be able to limit the total memory of memtables. More specific, the document says "a flush will be triggered if total mutable memtable size exceeds 90% of the limit". However, from the [implementation|https://github.com/facebook/rocksdb/blob/e8263dbdaad0546c54bddd01a8454c2e750a86c2/include/rocksdb/write_buffer_manager.h#L55] we could tell it's not following the document (write buffer could go up to 150% of the size), which is really a surprise.
# Based on the fact of #2 and #3, I suggest we work around by doing an internal computation and setting the size of {{Cache}} and {{WriteBufferManager}} accordingly: assume the memory limit is {{N}} and the write ratio is R, we have {{1.5*size_of_write_buffer_manager=R*size_of_cache}}, {{1.5*size_of_write_buffer_manager+size_of_others=N}}, and {{size_of_write_buffer_manager+size_of_others=size_of_cache}}, then we know {{size_of_write_buffer_manager=2NR/(3+R)}} and {{size_of_cache=3N/(3+R)}}
# For the additional cost of pinned iterator and/or index, it should be relatively small, and if it indeed cause out-of-memory, we suggest to use {{taskmanager.memory.task.offheap}} to cover this part.
# Note that we will be able to get rid of the kind-of-complicated work-around right after the rocksdb bug is fixed.
What do you think?
[~sewen] please also shed some lights here, thanks.
> 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)