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

Yun Tang commented on FLINK-15368:
----------------------------------

Updated progress:
 * Why memory cannot be controlled well under capacity?
 ** My previous experiments set 
[optimizeForPointLookup|https://github.com/facebook/rocksdb/blob/e8263dbdaad0546c54bddd01a8454c2e750a86c2/java/src/main/java/org/rocksdb/ColumnFamilyOptionsInterface.java#L25]
 by mistake and that option caused more memory usage in cache.
 ** Even without the {{optimizeForPointLookup}}, the write buffer manager would 
still exceed the capacity as write buffer would *not* be flushed if active 
mutable write buffer size is less than half of capacity of write buffer 
manager, please refer to [code 
here|https://github.com/facebook/rocksdb/blob/e8263dbdaad0546c54bddd01a8454c2e750a86c2/include/rocksdb/write_buffer_manager.h#L55].
 In other words, if we set a LRUCache of capacity 400MB and write buffer 
manager of capacity 200MB, we could use mem tables as 200*1.5=300MB which might 
lead us to exceed the total cache usage.
 ** Actually, there still exists another out-of-capacity risk since we would 
pin L0 index & filter in block cache.

 
 * Why java process would crash and core dump if we set LRUCache with 
{{strict_capacity_limit}} enabled?
 ** From my point of view, this is due to the behavior of costing write buffer 
memory usage to strict limited cache is still not deterministic. RocksDB lacks 
of any reasonable action if we fail to insert dummy entries into cache and I 
have already created a issue to report this problem: 
[RocksDB-issue/6247|https://github.com/facebook/rocksdb/issues/6247]. Besides, 
[RocksDB-pr/5175|https://github.com/facebook/rocksdb/pull/5175] could help to 
reduce the exceed of memory budget of LRUCache shard.

 
 * What shall we do to help mitigate the risk of out-of-capacity?
 ** I'm afraid we cannot rely on strict limited capacity cache in Flink-1.10 
before its final release. As I have described above for why write buffer could 
exceed capacity,  we could introduce a buffer space between off-heap memory 
capacity and actual block cache capacity to consider the extra half write 
buffer manager capacity. User could also configure this buffer space if extra 
memory in pinned iterator or index.

Last but not least, I noticed there exist obvious performance regression if we 
enable memory control for RocksDB. One main point is that write buffer would 
flush more frequently then before in small cache scenario (1GB TM would only 
have 300MB off-heap space, and we have 4 slots each TM in test that means 
rocksDB instances would only share a cache less than 80MB per slot). 
 The root cause is 
[arena_block_size|https://github.com/dataArtisans/frocksdb/blob/958f191d3f7276ae59b270f9db8390034d549ee0/db/column_family.cc#L196]
 is a bit large when we share cache among RocksDB instances. The arena block 
size would be set as 1/8 of write buffer size if we did not configure it. As 
the default write buffer size is 64MB, rocksDB would use 64/8=8MB to allocate 
memory if needed. As [write buffer manage 
doc|https://github.com/facebook/rocksdb/wiki/Write-Buffer-Manager#limit-total-memory-of-memtables]
 said: "In version 5.6 or higher, the memory is counted as total memory 
allocated in arena, even if some of them may not yet be used by memtable." 
Memtable is more easily to hit the limit when we use a quite larger arena 
block, however, the actual usage of memtable is only several KBs. One possible 
solution to resolve this is to decrease the arena block size explicitly. 

 

> 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
>
>
> 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)

Reply via email to