Hi Gabor,
Returning to your answer from a while ago. Since writing to you about
handling 10s GB of state with Apache Flink I am now deploying jobs with
multiple terabytes of state.
The problems that I outlined in my initial email have been handled by doing
some of the tuning that you suggested -
Okay, I see. Thanks for the explanation.
Br,
Bjarke
On Wed, Aug 21, 2024 at 10:17 AM Gabor Somogyi
wrote:
> There is no golden egg in the RocksDB tuning area but it can help a
> significantly.
>
> As a general rule the RocksDB state backend requires mainly 2 things to
> perform well, memory a
There is no golden egg in the RocksDB tuning area but it can help a
significantly.
As a general rule the RocksDB state backend requires mainly 2 things to
perform well, memory and fast local storage.
You can tune these depending on the bottleneck and make measurements.
Memory is used as a cache a
Hi Gabor,
Thanks for the suggestions.
For the rocksDB config tweaks would you be able to make some concrete
suggestions? I have looked into the official Tuning Checkpoints and Large
State guide documentation, however tuning rocksDB does not seem
straightforward tbh.
My entire flinkConfiguration
Please see my comments inline.
G
On Thu, Aug 8, 2024 at 3:41 PM Bjarke Tornager
wrote:
> Hi Gabor,
>
> What kind of development work - is it the Disaggregated State Storage and
> Management FLIP (
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=293046855)
> that you are refe
Hi Gabor,
What kind of development work - is it the Disaggregated State Storage and
Management FLIP (
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=293046855)
that you are referring to?
Thanks for the suggestion. I am not familiar with the state processor api
but I will have a
Hi Bjarke,
It's normal to see longer recovery time as the state size grows. There are
developments in progress to mitigate this problem.
> Any ideas as to what could be causing this?
I think there is no easy way to tell it, I can give just some pointers.
First I would take a look at the state fi
Hi,
I am doing stateful streaming (keyed coprocessing) using rocksDB. When my
job restarts to adjust parallelism, and restores from savepoint, I quite
often get the following error:
2024-08-07 19:43:09
java.lang.Exception: Exception while creating StreamOperatorStateContext.
at org.apache.flink.s