Hi Cam,

Which flink version are you using?

Actually I don't think any existing flink release could take usage of the
write buffer manager natively through some configuration magic, but
requires some "developing" efforts, such as manually building flink with a
higher version rocksdb to have the JNI interface to set write buffer
manager, and set the write buffer manager into rocksdb's DBOptions with a
custom options factory. More details please refer to this comment [1] in
FLINK-7289.

As mentioned in another thread [2], we are now working on removing all
these "manual steps" and making the state backend memory management "hands
free", which is also part of the FLIP-49 work. Hopefully we could get this
done in 1.10 release, let's see (smile).

[1] https://s.apache.org/5ay97
[2] https://s.apache.org/ej2zn

Best Regards,
Yu


On Fri, 9 Aug 2019 at 03:53, Congxian Qiu <qcx978132...@gmail.com> wrote:

> Hi
> Maybe FLIP-49[1] "Unified Memory Configuration for TaskExecutors" can give
> some information here
>
> [1]
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-49-Unified-Memory-Configuration-for-TaskExecutors-td31436.html
> Best,
> Congxian
>
>
> Cam Mach <cammac...@gmail.com> 于2019年8月9日周五 上午4:59写道:
>
>> Hi Biao, Yun and Ning.
>>
>> Thanks for your response and pointers. Those are very helpful!
>>
>> So far, we have tried with some of those parameters (WriterBufferManager,
>> write_buffer_size, write_buffer_count, ...), but still continuously having
>> issues with memory.
>> Here are our cluster configurations:
>>
>>    - 1 Job Controller (32 GB RAM and 8 cores)
>>    - 10 Task Managers: (32 GB RAM, 8 cores CPU, and 300GB SSD configured
>>    for RocksDB, and we set 10GB heap for each)
>>    - Running under Kuberntes
>>
>> We have a pipeline that read/transfer 500 million records (around 1kb
>> each), and write to our sink. Our total data is around 1.2 Terabytes. Our
>> pipeline configurations are as follows:
>>
>>    - 13 operators - some of them (around 6) are stateful
>>    - Parallelism: 60
>>    - Task slots: 6
>>
>> We have run several tests and observed that memory just keep growing
>> while our TM's CPU stay around 10 - 15% usage. We are now just focusing
>> limiting memory usage from Flink and RocksDB, so Kubernetes won't kill it.
>>
>> Any recommendations or advices are greatly appreciated!
>>
>> Thanks,
>>
>>
>>
>>
>> On Thu, Aug 8, 2019 at 6:57 AM Yun Tang <myas...@live.com> wrote:
>>
>>> Hi Cam
>>>
>>> I think FLINK-7289 [1] might offer you some insights to control RocksDB
>>> memory, especially the idea using write buffer manager [2] to control the
>>> total write buffer memory. If you do not have too many sst files, write
>>> buffer memory usage would consume much more space than index and filter
>>> usage. Since Flink would use per state per column family, and the write
>>> buffer number increase when more column families created.
>>>
>>>
>>> [1] https://issues.apache.org/jira/browse/FLINK-7289
>>> [2] https://github.com/dataArtisans/frocksdb/pull/4
>>>
>>> Best
>>> Yun Tang
>>>
>>>
>>> ------------------------------
>>> *From:* Cam Mach <cammac...@gmail.com>
>>> *Sent:* Thursday, August 8, 2019 21:39
>>> *To:* Biao Liu <mmyy1...@gmail.com>
>>> *Cc:* miki haiat <miko5...@gmail.com>; user <user@flink.apache.org>
>>> *Subject:* Re: Capping RocksDb memory usage
>>>
>>> Thanks for your response, Biao.
>>>
>>>
>>>
>>> On Wed, Aug 7, 2019 at 11:41 PM Biao Liu <mmyy1...@gmail.com> wrote:
>>>
>>> Hi Cam,
>>>
>>> AFAIK, that's not an easy thing. Actually it's more like a Rocksdb
>>> issue. There is a document explaining the memory usage of Rocksdb [1]. It
>>> might be helpful.
>>>
>>> You could define your own option to tune Rocksdb through
>>> "state.backend.rocksdb.options-factory" [2]. However I would suggest not to
>>> do this unless you are fully experienced of Rocksdb. IMO it's quite
>>> complicated.
>>>
>>> Meanwhile I can share a bit experience of this. We have tried to put the
>>> cache and filter into block cache before. It's useful to control the memory
>>> usage. But the performance might be affected at the same time. Anyway you
>>> could try and tune it. Good luck!
>>>
>>> 1. https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
>>> 2.
>>> https://ci.apache.org/projects/flink/flink-docs-master/ops/state/large_state_tuning.html#tuning-rocksdb
>>>
>>> Thanks,
>>> Biao /'bɪ.aʊ/
>>>
>>>
>>>
>>> On Thu, Aug 8, 2019 at 11:44 AM Cam Mach <cammac...@gmail.com> wrote:
>>>
>>> Yes, that is correct.
>>> Cam Mach
>>> Software Engineer
>>> E-mail: cammac...@gmail.com
>>> Tel: 206 972 2768
>>>
>>>
>>>
>>> On Wed, Aug 7, 2019 at 8:33 PM Biao Liu <mmyy1...@gmail.com> wrote:
>>>
>>> Hi Cam,
>>>
>>> Do you mean you want to limit the memory usage of RocksDB state backend?
>>>
>>> Thanks,
>>> Biao /'bɪ.aʊ/
>>>
>>>
>>>
>>> On Thu, Aug 8, 2019 at 2:23 AM miki haiat <miko5...@gmail.com> wrote:
>>>
>>> I think using metrics exporter is the easiest way
>>>
>>> [1]
>>> https://ci.apache.org/projects/flink/flink-docs-stable/monitoring/metrics.html#rocksdb
>>>
>>>
>>> On Wed, Aug 7, 2019, 20:28 Cam Mach <cammac...@gmail.com> wrote:
>>>
>>> Hello everyone,
>>>
>>> What is the most easy and efficiently way to cap RocksDb's memory usage?
>>>
>>> Thanks,
>>> Cam
>>>
>>>

Reply via email to