[
https://issues.apache.org/jira/browse/FLINK-11193?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Fan weiwen updated FLINK-11193:
-------------------------------
Description:
set state backends is rocksdb
set 'state.backend.rocksdb.timer-service.factory' by Configuration
{code:java}
RocksDBStateBackend backend = new RocksDBStateBackend(checkpointPath,true);
Configuration configuration = new Configuration();
configuration.setString(
RocksDBOptions.TIMER_SERVICE_FACTORY,
RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString());
backend = backend.configure(configuration);
{code}
then submit the job by client
but the job runtime the 'TIMER_SERVICE_FACTORY' Covered by jobmaster
jobmaster code
org.apache.flink.runtime.state.StateBackendLoader
{code:java}
backend = ((ConfigurableStateBackend) fromApplication).configure(config);
{code}
jobmaster buildGraph config is from flink-conf.yaml
Final execution of the same method
org.apache.flink.contrib.streaming.state.RocksDBStateBackend
{code:java}
final String priorityQueueTypeString = config.getString(TIMER_SERVICE_FACTORY);
this.priorityQueueStateType = priorityQueueTypeString.length() > 0 ?
PriorityQueueStateType.valueOf(priorityQueueTypeString.toUpperCase()) :
original.priorityQueueStateType;
{code}
the problem is config.getString(TIMER_SERVICE_FACTORY) always have value
regardless of configuration
so my customize config is
was:
set state backends is rocksdb
set 'state.backend.rocksdb.timer-service.factory' by Configuration
{code:java}
RocksDBStateBackend backend = new RocksDBStateBackend(checkpointPath,true);
Configuration configuration = new Configuration();
configuration.setString(
RocksDBOptions.TIMER_SERVICE_FACTORY,
RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString());
backend = backend.configure(configuration);
{code}
then submit the job by client
but the job runtime the 'TIMER_SERVICE_FACTORY' Covered by jobmaster
jobmaster code
org.apache.flink.runtime.state.StateBackendLoader
{code:java}
backend = ((ConfigurableStateBackend) fromApplication).configure(config);
{code}
jobmaster buildGraph config is from flink-conf.yaml
Final execution of the same method
org.apache.flink.contrib.streaming.state.RocksDBStateBackend
{code:java}
final String priorityQueueTypeString = config.getString(TIMER_SERVICE_FACTORY);
this.priorityQueueStateType = priorityQueueTypeString.length() > 0 ?
PriorityQueueStateType.valueOf(priorityQueueTypeString.toUpperCase()) :
original.priorityQueueStateType;
{code}
the problem is config.getString(TIMER_SERVICE_FACTORY) always have value
regardless of configuration
so my
> rockdb customize configuration Covered jobmaster
> ------------------------------------------------
>
> Key: FLINK-11193
> URL: https://issues.apache.org/jira/browse/FLINK-11193
> Project: Flink
> Issue Type: Bug
> Components: State Backends, Checkpointing
> Affects Versions: 1.6.2, 1.7.0
> Reporter: Fan weiwen
> Priority: Major
>
> set state backends is rocksdb
> set 'state.backend.rocksdb.timer-service.factory' by Configuration
> {code:java}
> RocksDBStateBackend backend = new RocksDBStateBackend(checkpointPath,true);
> Configuration configuration = new Configuration();
> configuration.setString(
> RocksDBOptions.TIMER_SERVICE_FACTORY,
> RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString());
> backend = backend.configure(configuration);
> {code}
> then submit the job by client
> but the job runtime the 'TIMER_SERVICE_FACTORY' Covered by jobmaster
> jobmaster code
> org.apache.flink.runtime.state.StateBackendLoader
> {code:java}
> backend = ((ConfigurableStateBackend) fromApplication).configure(config);
> {code}
> jobmaster buildGraph config is from flink-conf.yaml
> Final execution of the same method
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend
> {code:java}
> final String priorityQueueTypeString =
> config.getString(TIMER_SERVICE_FACTORY);
> this.priorityQueueStateType = priorityQueueTypeString.length() > 0 ?
> PriorityQueueStateType.valueOf(priorityQueueTypeString.toUpperCase()) :
> original.priorityQueueStateType;
> {code}
> the problem is config.getString(TIMER_SERVICE_FACTORY) always have value
> regardless of configuration
> so my customize config is
>
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)