[
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17133134#comment-17133134
]
Nico Kruber commented on FLINK-18242:
-------------------------------------
Thanks for the analysis - after changing to the new interface, the options
factory did indeed work as expected.
It seems that I got into this situation by compiling against Flink 1.9 and
executing in 1.10: Since I did not bundle the {{DefaultOptionsFactory}} class
from 1.9, at runtime Flink would only see the 1.10 file and interfaces and thus
missed my {{createDBOptions(DBOptions currentOptions)}}. Changing to compile
against 1.10 did not help me noticing the interface change since the old code
was still valid.
IMHO, this silent API change may be a problem (for some users): since there are
new getters/setters for the new {{RocksDBOptionsFactory}} and there is the
{{RocksDBOptionsFactoryAdapter}}, it would not have to inherit from
{{OptionsFactory}} to keep backwards compatibility. This may be a different
story for {{DefaultOptionsFactory}} (which may have needed both interfaces and
then special handling, or a {{final createDBOptions(DBOptions
currentOptions)}}, or just live with the API change) but neither of the
mentioned classes has an API annotation, not even {{@PublicEvolving}}.
# I would be against making {{DefaultConfigurableOptionsFactory}} final as this
does not solve anything and the class may actually be useful if you can tune
parameters via the cluster config and only need some additional manual settings
that are not exposed via config.
# just adding more documentation would probably not have helped here, at least
not for old code
> Custom OptionsFactory settings seem to have no effect on RocksDB
> ----------------------------------------------------------------
>
> Key: FLINK-18242
> URL: https://issues.apache.org/jira/browse/FLINK-18242
> Project: Flink
> Issue Type: Bug
> Components: Runtime / State Backends
> Affects Versions: 1.10.0, 1.10.1, 1.11.0
> Reporter: Nico Kruber
> Priority: Major
> Attachments: DefaultConfigurableOptionsFactoryWithLog.java
>
>
> When I configure a custom {{OptionsFactory}} for RocksDB like this
> (similarly by specifying it via the {{state.backend.rocksdb.options-factory}}
> configuration):
> {code:java}
> Configuration globalConfig = GlobalConfiguration.loadConfiguration();
> String checkpointDataUri =
> globalConfig.getString(CheckpointingOptions.CHECKPOINTS_DIRECTORY);
> RocksDBStateBackend stateBackend = new RocksDBStateBackend(checkpointDataUri);
> stateBackend.setOptions(new DefaultConfigurableOptionsFactoryWithLog());
> env.setStateBackend((StateBackend) stateBackend);{code}
> it seems to be loaded
> {code:java}
> 2020-06-10 12:54:20,720 INFO
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend - Using
> predefined options: DEFAULT.
> 2020-06-10 12:54:20,721 INFO
> org.apache.flink.contrib.streaming.state.RocksDBStateBackend - Using
> application-defined options factory:
> DefaultConfigurableOptionsFactoryWithLog{DefaultConfigurableOptionsFactory{configuredOptions={}}}.
> {code}
> but it seems like none of the options defined in there is actually used. Just
> as an example, my factory does set the info log level to {{INFO_LEVEL}} but
> this is what you will see in the created RocksDB instance:
> {code:java}
> > cat /tmp/flink-io-c95e8f48-0daa-4fb9-a9a7-0e4fb42e9135/*/db/OPTIONS*|grep
> > info_log_level
> info_log_level=HEADER_LEVEL
> info_log_level=HEADER_LEVEL{code}
> Together with the bug from FLINK-18241, it seems I cannot re-activate the
> RocksDB log that we disabled in FLINK-15068. FLINK-15747 was aiming at
> changing that particular configuration, but the problem seems broader since
> {{setDbLogDir()}} was actually also ignored and Flink itself does not change
> that setting.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)