Hi,
I'm benchmarking a job with large state in various window sizes (hourly, daily). I noticed that it would consistently slow down after 30 minutes into the benchmark due to high disk read IOPs. The first 30 minutes were fine, with close to 0 disk IOPs. Then after 30 minutes, read IOPs would gradually climb to as high as 10k/s. At this point, the job was bottlenecked on disk IOPs because I'm using 2TB EBS-backed volume. Another thread on the mailing list mentioned potentially running into burst IOPs credit could be the cause of slowdown. It's not that in this case because I'm using 2TB EBS. Someone also mentioned RocksDB compaction could potentially increase read IOPs a lot. I'm currently running the job with these RocksDB settings. @Override public DBOptions createDBOptions(DBOptions currentOptions) { return currentOptions .setIncreaseParallelism(4) .setUseFsync(false) .setMaxOpenFiles(-1); } @Override public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions currentOptions) { final long blockCacheSize = 64 * 1024 * 1024; return currentOptions .setTableFormatConfig( new BlockBasedTableConfig() .setBlockCacheSize(blockCacheSize) ); } Any insights into how I can further diagnose this? Is there anyway to see compaction stats or any settings I should try? Thanks, Ning |
Hi Ning
From your description, I think you actually concern more about the overall performance instead of the high disk IOPs. Maybe you should first ensure whether the job performance degradation is related to RocksDB's performance.
Then I would share some experience about tuning RocksDB performance. Since you did not cache index and filter in block cache, it's no worry about the competition between data blocks and index&filter blocks[1]. And to improve the read performance, you should
increase your block cache size to 256MB or even 512MB. What's more, writer buffer in rocksDB also acts as a role for reading, from our experience, we use 4 max write buffers and 32MB each, e.g. setMaxWriteBufferNumber(4) and setWriteBufferSize(32*1024*1024)
Best
Yun Tang
From: Ning Shi <[hidden email]>
Sent: Wednesday, September 26, 2018 11:25 To: user Subject: RocksDB Read IOPs Hi,
I'm benchmarking a job with large state in various window sizes (hourly, daily). I noticed that it would consistently slow down after 30 minutes into the benchmark due to high disk read IOPs. The first 30 minutes were fine, with close to 0 disk IOPs. Then after 30 minutes, read IOPs would gradually climb to as high as 10k/s. At this point, the job was bottlenecked on disk IOPs because I'm using 2TB EBS-backed volume. Another thread on the mailing list mentioned potentially running into burst IOPs credit could be the cause of slowdown. It's not that in this case because I'm using 2TB EBS. Someone also mentioned RocksDB compaction could potentially increase read IOPs a lot. I'm currently running the job with these RocksDB settings. @Override public DBOptions createDBOptions(DBOptions currentOptions) { return currentOptions .setIncreaseParallelism(4) .setUseFsync(false) .setMaxOpenFiles(-1); } @Override public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions currentOptions) { final long blockCacheSize = 64 * 1024 * 1024; return currentOptions .setTableFormatConfig( new BlockBasedTableConfig() .setBlockCacheSize(blockCacheSize) ); } Any insights into how I can further diagnose this? Is there anyway to see compaction stats or any settings I should try? Thanks, Ning |
Hi Yun,
This is very helpful. I did try increasing the block cache to 256MB or 512MB. It quickly used up the 30GB memory on the EC2 instances. I found it a little hard to estimate the actual memory usage of RocksDB as there might be multiple instances on the same TM depending on the number of slots and job. In this case, each instance has 16 cores and 30GB of memory. Each TM has 8 slots. The job parallelism equals to the total number of slots across all TMs so they use all the slots. Heap is set to 5GB. With 64MB block cache size, the memory usage seems to hover around 20GB to 25GB, but it creeps up very slowly over time. Do you have a good strategy for memory usage estimation or recommendation for how much memory each instance should have? Thanks, Ning
|
In reply to this post by Yun Tang
Yun,
> Then I would share some experience about tuning RocksDB performance. Since you did not cache index and filter in block cache, it's no worry about the competition between data blocks and index&filter blocks[1]. And to improve the read performance, you should increase your block cache size to 256MB or even 512MB. What's more, writer buffer in rocksDB also acts as a role for reading, from our experience, we use 4 max write buffers and 32MB each, e.g. setMaxWriteBufferNumber(4) and setWriteBufferSize(32*1024*1024) Thank you very much for the hints. I read that tuning guide and added some settings. Now it's doing much much better. The IOPs stays under 300 except for when checkpoints are taken, then it spikes to about 1.5k, which is totally expected. For reference, the following are the settings I'm using right now. The reason I didn't bump block cache size is because we have limited amount of memory per instance (30GB). @Override public DBOptions createDBOptions(DBOptions currentOptions) { return currentOptions .setIncreaseParallelism(4) .setMaxBackgroundFlushes(1) .setMaxBackgroundCompactions(1) .setUseFsync(false) .setMaxOpenFiles(-1); } @Override public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions currentOptions) { final long blockCacheSize = 64 * 1024 * 1024; final long writeBufferSize = 64 * 1024 * 1024; return currentOptions .setCompressionType(CompressionType.LZ4_COMPRESSION) .setCompactionStyle(CompactionStyle.LEVEL) .setLevel0FileNumCompactionTrigger(10) .setLevel0SlowdownWritesTrigger(20) .setLevel0StopWritesTrigger(40) .setWriteBufferSize(writeBufferSize) // In-memory memtable size .setMaxWriteBufferNumber(5) // Max number of memtables before stalling writes .setMinWriteBufferNumberToMerge(2) // Merge two memtables together to reduce duplicate keys .setTargetFileSizeBase(writeBufferSize) // L0 file size, same as memtable size .setMaxBytesForLevelBase(writeBufferSize * 8) .setTableFormatConfig( new BlockBasedTableConfig() .setFilter(new BloomFilter()) .setBlockCacheSize(blockCacheSize) ); } Ning |
Free forum by Nabble | Edit this page |