You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@flink.apache.org by Thomas Weise <th...@apache.org> on 2018/11/09 17:07:40 UTC

How to use RocksDBStateBackend predefined options

Hi,

We are observing unbounded memory consumption with the default
RocksDBStateBackend configuration [1]. According to [2] it is possible to
control that through custom options or by using the predefined options from
[3].

Is there a way to activate the predefined options via configuration / flink-
conf.yaml? Or only programmatically, like in [4]? The difficulty with the
programmatic route (assuming this works now), is that in my case the client
is Beam and I'm not writing the code that submits the job.

Thanks,
Thomas

[1] https://stackoverflow.com/questions/50812837/flink-taskmanager
-out-of-memory-and-memory-configuration/53229810#53229810

[2] https://ci.apache.org/projects/flink/flink
-docs-stable/ops/state/large_state_tuning.html#tuning-rocksdb

[3] https://github.com/apache/flink/blob/release-1.5/flink-state-backends/
flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib
/streaming/state/PredefinedOptions.java#L44

[4] https://stackoverflow.com/questions/44819521/how-to-set-
rocksdbstatebackend-parameter-in-flink

Re: How to use RocksDBStateBackend predefined options

Posted by Thomas Weise <th...@apache.org>.
Done: https://issues.apache.org/jira/browse/FLINK-10912

Thanks,
Thomas


On Thu, Nov 15, 2018 at 1:22 AM Till Rohrmann <tr...@apache.org> wrote:

> I think it sound like a good idea to be able to specify an options factory
> in the flink-conf.yaml. Please go ahead with creating the respective JIRA
> issues.
>
> Cheers,
> Till
>
> On Wed, Nov 14, 2018 at 7:33 AM Yun Tang <my...@live.com> wrote:
>
> > Hi all
> >
> > We already found the programmatic way to configure RocksDB was not so
> > friendly in our Alibaba's production environment, and refactored it by
> > wrapping customer configurations for RocksDB. We pre-wrapped
> configurations
> > such as block cache size, whether to cache index&filter into block cache
> > size, number of background flush threads, and could get the them from
> > TaskManagerConfiguration if users want to change the default values. But
> I
> > think our community's way using options factory could also make sense if
> > sophisticated user want to make fine-grained tuning which might use
> > configuration not included in our pre-wrapped configuration. We might
> > combine these two kind of operations together.
> >
> > I'd like to create a JIRA about this improvement, what's your opinions?
> >
> > Best
> > Yun Tang
> >
> > ________________________________
> > From: Thomas Weise <th...@apache.org>
> > Sent: Tuesday, November 13, 2018 1:53
> > To: dev@flink.apache.org
> > Cc: uce@apache.org
> > Subject: Re: How to use RocksDBStateBackend predefined options
> >
> > Sounds good. Perhaps it would also be good to allow the user to specify
> an
> > options factory in flink-conf.yaml for more flexibility?
> >
> > Thomas
> >
> > On Mon, Nov 12, 2018 at 9:48 AM Stefan Richter <
> > s.richter@data-artisans.com>
> > wrote:
> >
> > > Hi,
> > >
> > > Ufuk is right, for historical reasons there is currently only the
> > > programatic way but I think nothing speaks fundamentally against
> offering
> > > configuration via config in the future (maybe just a lot of config keys
> > > must be introduced to cover all options).
> > >
> > > Best,
> > > Stefan
> > >
> > > > On 9. Nov 2018, at 22:52, Ufuk Celebi <uc...@apache.org> wrote:
> > > >
> > > > Hey Thomas,
> > > >
> > > > On Fri, Nov 9, 2018 at 6:07 PM Thomas Weise <th...@apache.org> wrote:
> > > >> Is there a way to activate the predefined options via configuration
> /
> > > flink-
> > > >> conf.yaml? Or only programmatically, like in [4]? The difficulty
> with
> > > the
> > > >> programmatic route (assuming this works now), is that in my case the
> > > client
> > > >> is Beam and I'm not writing the code that submits the job.
> > > >
> > > > AFAIK no. You can only do it programmatically at the moment [1].
> > > > Having the option to configure all settings through the configuration
> > > > file seems to be a valid feature request to me.
> > > >
> > > > @Stefan Richter (cc'd): Are there any reasons that speak against
> > > > exposing these options via the configuration file in your opinion?
> > > >
> > > > Best,
> > > >
> > > > Ufuk
> > > >
> > > > [1] Looking at the code in [2] and [3], the only options that are
> > > > exposed via the configuration file are local directories and the
> timer
> > > > service factory.
> > > > [2]
> > >
> >
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java
> > > > [3]
> > >
> >
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
> > >
> > >
> >
>

Re: How to use RocksDBStateBackend predefined options

Posted by Till Rohrmann <tr...@apache.org>.
I think it sound like a good idea to be able to specify an options factory
in the flink-conf.yaml. Please go ahead with creating the respective JIRA
issues.

Cheers,
Till

On Wed, Nov 14, 2018 at 7:33 AM Yun Tang <my...@live.com> wrote:

> Hi all
>
> We already found the programmatic way to configure RocksDB was not so
> friendly in our Alibaba's production environment, and refactored it by
> wrapping customer configurations for RocksDB. We pre-wrapped configurations
> such as block cache size, whether to cache index&filter into block cache
> size, number of background flush threads, and could get the them from
> TaskManagerConfiguration if users want to change the default values. But I
> think our community's way using options factory could also make sense if
> sophisticated user want to make fine-grained tuning which might use
> configuration not included in our pre-wrapped configuration. We might
> combine these two kind of operations together.
>
> I'd like to create a JIRA about this improvement, what's your opinions?
>
> Best
> Yun Tang
>
> ________________________________
> From: Thomas Weise <th...@apache.org>
> Sent: Tuesday, November 13, 2018 1:53
> To: dev@flink.apache.org
> Cc: uce@apache.org
> Subject: Re: How to use RocksDBStateBackend predefined options
>
> Sounds good. Perhaps it would also be good to allow the user to specify an
> options factory in flink-conf.yaml for more flexibility?
>
> Thomas
>
> On Mon, Nov 12, 2018 at 9:48 AM Stefan Richter <
> s.richter@data-artisans.com>
> wrote:
>
> > Hi,
> >
> > Ufuk is right, for historical reasons there is currently only the
> > programatic way but I think nothing speaks fundamentally against offering
> > configuration via config in the future (maybe just a lot of config keys
> > must be introduced to cover all options).
> >
> > Best,
> > Stefan
> >
> > > On 9. Nov 2018, at 22:52, Ufuk Celebi <uc...@apache.org> wrote:
> > >
> > > Hey Thomas,
> > >
> > > On Fri, Nov 9, 2018 at 6:07 PM Thomas Weise <th...@apache.org> wrote:
> > >> Is there a way to activate the predefined options via configuration /
> > flink-
> > >> conf.yaml? Or only programmatically, like in [4]? The difficulty with
> > the
> > >> programmatic route (assuming this works now), is that in my case the
> > client
> > >> is Beam and I'm not writing the code that submits the job.
> > >
> > > AFAIK no. You can only do it programmatically at the moment [1].
> > > Having the option to configure all settings through the configuration
> > > file seems to be a valid feature request to me.
> > >
> > > @Stefan Richter (cc'd): Are there any reasons that speak against
> > > exposing these options via the configuration file in your opinion?
> > >
> > > Best,
> > >
> > > Ufuk
> > >
> > > [1] Looking at the code in [2] and [3], the only options that are
> > > exposed via the configuration file are local directories and the timer
> > > service factory.
> > > [2]
> >
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java
> > > [3]
> >
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
> >
> >
>

Re: How to use RocksDBStateBackend predefined options

Posted by Yun Tang <my...@live.com>.
Hi all

We already found the programmatic way to configure RocksDB was not so friendly in our Alibaba's production environment, and refactored it by wrapping customer configurations for RocksDB. We pre-wrapped configurations such as block cache size, whether to cache index&filter into block cache size, number of background flush threads, and could get the them from TaskManagerConfiguration if users want to change the default values. But I think our community's way using options factory could also make sense if sophisticated user want to make fine-grained tuning which might use configuration not included in our pre-wrapped configuration. We might combine these two kind of operations together.

I'd like to create a JIRA about this improvement, what's your opinions?

Best
Yun Tang

________________________________
From: Thomas Weise <th...@apache.org>
Sent: Tuesday, November 13, 2018 1:53
To: dev@flink.apache.org
Cc: uce@apache.org
Subject: Re: How to use RocksDBStateBackend predefined options

Sounds good. Perhaps it would also be good to allow the user to specify an
options factory in flink-conf.yaml for more flexibility?

Thomas

On Mon, Nov 12, 2018 at 9:48 AM Stefan Richter <s....@data-artisans.com>
wrote:

> Hi,
>
> Ufuk is right, for historical reasons there is currently only the
> programatic way but I think nothing speaks fundamentally against offering
> configuration via config in the future (maybe just a lot of config keys
> must be introduced to cover all options).
>
> Best,
> Stefan
>
> > On 9. Nov 2018, at 22:52, Ufuk Celebi <uc...@apache.org> wrote:
> >
> > Hey Thomas,
> >
> > On Fri, Nov 9, 2018 at 6:07 PM Thomas Weise <th...@apache.org> wrote:
> >> Is there a way to activate the predefined options via configuration /
> flink-
> >> conf.yaml? Or only programmatically, like in [4]? The difficulty with
> the
> >> programmatic route (assuming this works now), is that in my case the
> client
> >> is Beam and I'm not writing the code that submits the job.
> >
> > AFAIK no. You can only do it programmatically at the moment [1].
> > Having the option to configure all settings through the configuration
> > file seems to be a valid feature request to me.
> >
> > @Stefan Richter (cc'd): Are there any reasons that speak against
> > exposing these options via the configuration file in your opinion?
> >
> > Best,
> >
> > Ufuk
> >
> > [1] Looking at the code in [2] and [3], the only options that are
> > exposed via the configuration file are local directories and the timer
> > service factory.
> > [2]
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java
> > [3]
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
>
>

Re: How to use RocksDBStateBackend predefined options

Posted by Thomas Weise <th...@apache.org>.
Sounds good. Perhaps it would also be good to allow the user to specify an
options factory in flink-conf.yaml for more flexibility?

Thomas

On Mon, Nov 12, 2018 at 9:48 AM Stefan Richter <s....@data-artisans.com>
wrote:

> Hi,
>
> Ufuk is right, for historical reasons there is currently only the
> programatic way but I think nothing speaks fundamentally against offering
> configuration via config in the future (maybe just a lot of config keys
> must be introduced to cover all options).
>
> Best,
> Stefan
>
> > On 9. Nov 2018, at 22:52, Ufuk Celebi <uc...@apache.org> wrote:
> >
> > Hey Thomas,
> >
> > On Fri, Nov 9, 2018 at 6:07 PM Thomas Weise <th...@apache.org> wrote:
> >> Is there a way to activate the predefined options via configuration /
> flink-
> >> conf.yaml? Or only programmatically, like in [4]? The difficulty with
> the
> >> programmatic route (assuming this works now), is that in my case the
> client
> >> is Beam and I'm not writing the code that submits the job.
> >
> > AFAIK no. You can only do it programmatically at the moment [1].
> > Having the option to configure all settings through the configuration
> > file seems to be a valid feature request to me.
> >
> > @Stefan Richter (cc'd): Are there any reasons that speak against
> > exposing these options via the configuration file in your opinion?
> >
> > Best,
> >
> > Ufuk
> >
> > [1] Looking at the code in [2] and [3], the only options that are
> > exposed via the configuration file are local directories and the timer
> > service factory.
> > [2]
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java
> > [3]
> https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java
>
>

Re: How to use RocksDBStateBackend predefined options

Posted by Stefan Richter <s....@data-artisans.com>.
Hi,

Ufuk is right, for historical reasons there is currently only the programatic way but I think nothing speaks fundamentally against offering configuration via config in the future (maybe just a lot of config keys must be introduced to cover all options).

Best,
Stefan

> On 9. Nov 2018, at 22:52, Ufuk Celebi <uc...@apache.org> wrote:
> 
> Hey Thomas,
> 
> On Fri, Nov 9, 2018 at 6:07 PM Thomas Weise <th...@apache.org> wrote:
>> Is there a way to activate the predefined options via configuration / flink-
>> conf.yaml? Or only programmatically, like in [4]? The difficulty with the
>> programmatic route (assuming this works now), is that in my case the client
>> is Beam and I'm not writing the code that submits the job.
> 
> AFAIK no. You can only do it programmatically at the moment [1].
> Having the option to configure all settings through the configuration
> file seems to be a valid feature request to me.
> 
> @Stefan Richter (cc'd): Are there any reasons that speak against
> exposing these options via the configuration file in your opinion?
> 
> Best,
> 
> Ufuk
> 
> [1] Looking at the code in [2] and [3], the only options that are
> exposed via the configuration file are local directories and the timer
> service factory.
> [2] https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java
> [3] https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java


Re: How to use RocksDBStateBackend predefined options

Posted by Ufuk Celebi <uc...@apache.org>.
Hey Thomas,

On Fri, Nov 9, 2018 at 6:07 PM Thomas Weise <th...@apache.org> wrote:
> Is there a way to activate the predefined options via configuration / flink-
> conf.yaml? Or only programmatically, like in [4]? The difficulty with the
> programmatic route (assuming this works now), is that in my case the client
> is Beam and I'm not writing the code that submits the job.

AFAIK no. You can only do it programmatically at the moment [1].
Having the option to configure all settings through the configuration
file seems to be a valid feature request to me.

@Stefan Richter (cc'd): Are there any reasons that speak against
exposing these options via the configuration file in your opinion?

Best,

Ufuk

[1] Looking at the code in [2] and [3], the only options that are
exposed via the configuration file are local directories and the timer
service factory.
[2] https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBOptions.java
[3] https://github.com/apache/flink/blob/master/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackend.java