Hello everyone,
What is the most easy and efficiently way to cap RocksDb's memory usage? Thanks, Cam |
I think using metrics exporter is the easiest way On Wed, Aug 7, 2019, 20:28 Cam Mach <[hidden email]> 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 <[hidden email]> wrote:
|
On Wed, Aug 7, 2019 at 8:33 PM Biao Liu <[hidden email]> 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! On Thu, Aug 8, 2019 at 11:44 AM Cam Mach <[hidden email]> wrote:
|
Thanks for your response, Biao. On Wed, Aug 7, 2019 at 11:41 PM Biao Liu <[hidden email]> wrote:
|
In reply to this post by Cam Mach
Hi Cam,
This blog post has some pointers in tuning RocksDB memory usage that might be of help. https://klaviyo.tech/flinkperf-c7bd28acc67 Ning On Wed, Aug 7, 2019 at 1:28 PM Cam Mach <[hidden email]> wrote: > > Hello everyone, > > What is the most easy and efficiently way to cap RocksDb's memory usage? > > Thanks, > Cam > |
In reply to this post by Cam Mach
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.
Best
Yun Tang
From: Cam Mach <[hidden email]>
Sent: Thursday, August 8, 2019 21:39 To: Biao Liu <[hidden email]> Cc: miki haiat <[hidden email]>; user <[hidden email]> Subject: Re: Capping RocksDb memory usage Thanks for your response, Biao.
On Wed, Aug 7, 2019 at 11:41 PM Biao Liu <[hidden email]> wrote:
|
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:
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:
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 <[hidden email]> wrote:
|
Hi Maybe FLIP-49[1] "Unified Memory Configuration for TaskExecutors" can give some information here Cam Mach <[hidden email]> 于2019年8月9日周五 上午4:59写道:
|
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). Best Regards,
Yu On Fri, 9 Aug 2019 at 03:53, Congxian Qiu <[hidden email]> wrote:
|
Hi Yu, Yes, we recompiled Flink with rocksdb to have JNI, to enable the write_buffer_manager after we read that Jira. One quick question, I noticed that our disk usage (SSD) for RocksDb is always stay around %2 (or 2.2 GB), which is not the case before we enable RocksDb state backend. So wondering what stoping it? Thanks, Cam On Fri, Aug 9, 2019 at 12:21 AM Yu Li <[hidden email]> wrote:
|
bq. Yes, we recompiled Flink with rocksdb to have JNI, to enable the write_buffer_manager after we read that Jira. I see, then which way are you using to limit the rocksdb memory? Setting write buffer and block cache size separately or with the "cost memory used in memtable into block cache" [1] feature? If the latter one, please make sure you also have this PR [2] in your customized rocksdb.bq. I noticed that our disk usage (SSD) for RocksDb is always stay around %2 (or 2.2 GB), which is not the case before we enable RocksDb state backend Along with the state data ingestion as well as checkpoint execution, RocksDB state backend will flush sst files out onto local disk (along with a file uploading to HDFS when checkpointing). For heap backend, all data resident in memory, and write directly onto HDFS when checkpoint triggered, thus no local disk space usage. What's more, notice that if you enable local recovery (check whether "state.backend.local-recovery" is set to true in your configuration, by default it's false), there'll be more disk space occupation, but in this case both heap and rocksdb backend have the cost. On Fri, 9 Aug 2019 at 15:10, Cam Mach <[hidden email]> wrote:
|
Free forum by Nabble | Edit this page |