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

Stephan Ewen commented on FLINK-15368:
--------------------------------------

Thank you very much for the thorough investigation. That is super helpful!
 Some thoughts on immediate and long-term strategy blow.

 * +1 for the described computation as a workaround for the write buffer memory 
usage. Let's hope that the two changes Yun pointed out make it into RocksDB 
soon, so we can drop this workaround.

 * +1 to keep pinning index and filter blocks. This seems too crucial and we 
should document that this is a known case where the memory budget can in fact 
be exceeded.

 * The new model can lead to situations where out of the box less memory being 
dedicated to RocksDB, compared to before. If less memory is dedicated than 
before, it makes sense that this leads to a performance regression. Again, 
something to mention in the release notes.
If we increase the managed memory to match the previously dedicated memory, 
would we still expect a performance regression.

Questions:

(Q1): Is there a reason why we do not set the write buffer manager ratio to a 
higher value by default (0.9)? The way I understand it, it will not reserve 
that much memory from the cache eagerly (write buffers will still be flushed 
when full) but that value is only an upper bound. I guess that depends on what 
is more critical for performance: Fewer write buffer flushes or more cache hits.

(Q2): All the above sounds to me like it would be beneficial to have fewer 
column families in total, because that would lead to fewer write buffers and 
fewer index blocks and bloom filters. After the release, should we pick the 
discussion again to have one column family per task, see if that could help us


> 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