Details
-
Bug
-
Status: Closed
-
Minor
-
Resolution: Fixed
-
1.6.2, 1.6.4, 1.7.0
Description
set state backends is rocksdb
set 'state.backend.rocksdb.timer-service.factory' by Configuration
RocksDBStateBackend backend = new RocksDBStateBackend(checkpointPath,true); Configuration configuration = new Configuration(); configuration.setString( RocksDBOptions.TIMER_SERVICE_FACTORY, RocksDBStateBackend.PriorityQueueStateType.ROCKSDB.toString()); backend = backend.configure(configuration);
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
backend = ((ConfigurableStateBackend) fromApplication).configure(config);
jobmaster buildGraph config is from flink-conf.yaml
Final execution of the same method
org.apache.flink.contrib.streaming.state.RocksDBStateBackend
final String priorityQueueTypeString = config.getString(TIMER_SERVICE_FACTORY); this.priorityQueueStateType = priorityQueueTypeString.length() > 0 ? PriorityQueueStateType.valueOf(priorityQueueTypeString.toUpperCase()) : original.priorityQueueStateType;
the problem is config.getString(TIMER_SERVICE_FACTORY) always have value regardless of configuration
so my customize config is Covered by jobmaster
Attachments
Issue Links
- fixes
-
FLINK-10372 There is no API to configure the timer state backend
- Closed
- is duplicated by
-
FLINK-11192 rockdb customize configuration Covered jobmaster
- Closed
-
FLINK-10098 Programmatically select timer storage backend
- Closed
- links to