Hi Trystan,

You can use async-profiler[1] to detect the CPU stack within RocksDB to see 
what happened, maybe you can try to enable partitioned index & filters[2] if 
the call stack is occupied by loading index or filter block.

[1] https://github.com/jvm-profiling-tools/async-profiler
[2] 
https://nightlies.apache.org/flink/flink-docs-release-1.14/docs/deployment/config/#state-backend-rocksdb-memory-partitioned-index-filters

Best
Yun Tang


________________________________
From: Yaroslav Tkachenko <yaros...@goldsky.io>
Sent: Thursday, April 21, 2022 0:44
To: Trystan <entro...@gmail.com>
Cc: user <user@flink.apache.org>
Subject: Re: RocksDB efficiency and keyby

Yep, I'd give it another try. EBS could be too slow in some use-cases.

On Wed, Apr 20, 2022 at 9:39 AM Trystan 
<entro...@gmail.com<mailto:entro...@gmail.com>> wrote:
Thanks for the info! We're running EBS gp2 volumes... awhile back we tested 
local SSDs with a different job and didn't notice any gains, but that was 
likely due to an under-optimized job where the bottleneck was elsewhere

On Wed, Apr 20, 2022, 11:08 AM Yaroslav Tkachenko 
<yaros...@goldsky.io<mailto:yaros...@goldsky.io>> wrote:
Hey Trystan,

Based on my personal experience, good disk IO for RocksDB matters a lot. Are 
you using the fastest SSD storage you can get for RocskDB folders?

For example, when running on GCP, we noticed 10x throughput improvement by 
switching RocksDB storage to 
https://cloud.google.com/compute/docs/disks/local-ssd

On Wed, Apr 20, 2022 at 8:50 AM Trystan 
<entro...@gmail.com<mailto:entro...@gmail.com>> wrote:
Hello,

We have a job where its main purpose is to track whether or not we've 
previously seen a particular event - that's it. If it's new, we save it to an 
external database. If we've seen it, we block the write. There's a 3-day TTL to 
manage the state size. The downstream db can tolerate new data slipping through 
and reject the write - we mainly use the state to reduce writes.

We're starting to see some performance issues, even after adding 50% capacity 
to the job. After some number of days/weeks, it eventually goes into a constant 
backpressure situation. I'm wondering if there's something we can do to improve 
efficiency.

1. According to the flamegraph, 60-70% of the time is spent in RocksDB.get
2. The state is just a ValueState<Boolean>. I assume this is the smallest/most 
efficient state. The keyby is extremely high cardinality - are we better off 
with a lower cardinality and a MapState<String, Boolean> .contains() check?
3. Current configs: taskmanager.memory.process.size: 4g, 
taskmanager.memory.managed.fraction: 0.8 (increased from 0.6, didn't see much 
change)
4. Estimated num keys tops out somewhere around 9-10B. Estimated live data size 
somewhere around 250 GB. Attempting to switch to heap state immediately ran 
into OOM (parallelism: 120, 8gb memory each).

And perhaps the answer is just "scale out" :) but if there are any signals to 
know when we've reached the limit of current scale, it'd be great to know what 
signals to look for!

Thanks!
Trystan

Reply via email to