Hi, 如唐云所述,FLINK-7289 [1] 所有的开发工作已经完成,目前剩余的工作是补充end-to-end测试以及完善文档,因此release-1.10分支的代码功能已经完全可用了
我们建议使用FLINK-7289实现的方式来控制rocksdb内存,这将极大的简化用户所需的配置,只需要设置"state.backend.rocksdb.memory.managed"为true并调整managed memory大小,或者通过"state.backend.rocksdb.memory.fixed-per-slot" 配置对应单个slot RocksDB可使用的最大内存即可 如果生产上确实比较紧急,无法等待1.10.0版本的发布,也可以参考之前英文邮件列表里相关讨论 [2] 给出的公式和设置来尝试对rocksdb内存进行限制 希望这些信息有所帮助 Best Regards, Yu [1] https://issues.apache.org/jira/browse/FLINK-7289 [2] http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Memory-constrains-running-Flink-on-Kubernetes-tt28954.html On Tue, 14 Jan 2020 at 11:15, Yun Tang <myas...@live.com> wrote: > Hi Dong > > RocksDB无论如何都是要使用native内存的,您的YARN pmem-check相比JVM heap的buffer空间是多少,是否合适呢? > > FLINK-7289的基本所需task都已经完成在release-1.10 分支中了,您可以直接使用release-1.10 > 分支打包,最近也要发布1.10的rc版本,欢迎试用该功能。 > > 如果你的所有checkpoint size是50GB,其实不是很大,但是如果单个state > backend有50GB的话,对于Flink这种低延迟流式场景是稍大的,建议降低单并发state数据量。 > > 至于目前的问题,也就是您加了相关参数,但是内存用量仍然在涨,可以用以下思路排查一下: > > 1. 首先,确保使用release-1.10 分支 > 2. 开启 size-all-mem-tables [1] 和 block-cache-usage [2]的metrics监控 > 3. 在默认没有enable "state.backend.rocksdb.memory.managed" [3] 的情况下,对column > family进行如下配置,核心思路就是将主要的内存使用都放在cache中,方便观察内存使用: > > rocksDBStateBackend.setOptions(new OptionsFactory() { > @Override > public DBOptions createDBOptions(DBOptions currentOptions) { > return currentOptions; > } > > @Override > public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions > currentOptions) { > BlockBasedTableConfig blockBasedTableConfig = new > BlockBasedTableConfig(); > blockBasedTableConfig.setCacheIndexAndFilterBlocks(true); > blockBasedTableConfig.pinL0FilterAndIndexBlocksInCache(); > currentOptions.setTableFormatConfig(blockBasedTableConfig); > return currentOptions; > } > }); > > 4. 由于没有enable cache共享,所以需要将每个column > family的size-all-mem-tables和block-cache-usage进行相加,观察相关指数变化,看是否超过了你的pmem-check > 限制。 > > 相应地,您也可以启用"state.backend.rocksdb.memory.managed" [3] 该功能 或者 自行配置 > "state.backend.rocksdb.memory.fixed-per-slot" [4] 设置期望的rocksDB per slot > memory size,此时只需要观察block-cache-usage的指标,由于这里使用共享cache的逻辑,所以并不需要相加,只要观察per > slot的情况即可(同一个TM内,相同subtask index的rocksDB state其实是用的同一块cache),观察内存限制功能是否生效。 > > [1] > https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/config.html#state-backend-rocksdb-metrics-size-all-mem-tables > [2] > https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/config.html#state-backend-rocksdb-metrics-block-cache-usage > [3] > https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/config.html#state-backend-rocksdb-memory-managed > [4] > https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/config.html#state-backend-rocksdb-memory-fixed-per-slot > > > 祝好 > 唐云 > > ________________________________ > From: DONG, Weike <kyled...@connect.hku.hk> > Sent: Tuesday, January 14, 2020 10:02 > To: user-zh@flink.apache.org <user-zh@flink.apache.org> > Subject: 咨询一下 RocksDB 状态后端的调优经验 > > 大家好, > > 我们在 YARN 容器内运行以 RocksDB 作为 State Backend 的 Flink 作业,状态数据比较大(50G > 以上,难以放到内存中)。但是由于 YARN 本身的 pmem-check 限制,经常会因为内存用量的不受控而导致整个 Container 被强制 > KILL. > > 目前调研了 https://issues.apache.org/jira/browse/FLINK-7289 这个提议,但是目前还未完全实现。 > 也按照 RocksDB 官方的调优指南 > https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide 设置了 > state.backend.rocksdb.writebuffer.size > state.backend.rocksdb.writebuffer.count > state.backend.rocksdb.block.cache-size > state.backend.rocksdb.files.open > 等等参数,但是目前观察到效果并不太明显,内存用量还是会不受控地越来越多。 > > 请问各位是否有 RocksDB 作为状态后端的调优经验,例如在内存受限的情况下,尽量确保 RocksDB 的内存用量可控在一个封顶范围呢? > > 另外还有一个场景,假设内存够用的情况下,有哪些增加读写性能方面的建议呢?目前尝试使用 SSD 来存放 sst 文件,但是性能提升也不明显。 > > 感谢 :) >