[jira] [Commented] (FLINK-18242) Custom OptionsFactory settings seem to have no effect on RocksDB

2020-06-17 Thread Yu Li (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17138459#comment-17138459
 ] 

Yu Li commented on FLINK-18242:
---

Merged into master via 5ed371fb17ee842240dbb886bf5fdefab6d9db62

> 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
>Assignee: Yu Li
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.12.0
>
> 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)


[jira] [Commented] (FLINK-18242) Custom OptionsFactory settings seem to have no effect on RocksDB

2020-06-15 Thread Yu Li (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17136294#comment-17136294
 ] 

Yu Li commented on FLINK-18242:
---

bq. For 1.11 / 1.12 what do you think about dropping the OptionsFactory and 
only go ahead with the RocksDBOptionsFactory?

+1, let me prepare the PRs separately.

> 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
>Assignee: Yu Li
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.12.0
>
> 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)


[jira] [Commented] (FLINK-18242) Custom OptionsFactory settings seem to have no effect on RocksDB

2020-06-15 Thread Stephan Ewen (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17135986#comment-17135986
 ] 

Stephan Ewen commented on FLINK-18242:
--

I see this is a thing we need to fix in 1.10.x.

For 1.11 / 1.12 what do you think about dropping the {{OptionsFactory}} and 
only go ahead with the {{RocksDBOptionsFactory}}.
That would be less confusing for the users and we had the {{OptionsFactory}} 
deprecated in the previous release.


> 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
>Assignee: Yu Li
>Priority: Major
>  Labels: pull-request-available
> Fix For: 1.12.0
>
> 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)


[jira] [Commented] (FLINK-18242) Custom OptionsFactory settings seem to have no effect on RocksDB

2020-06-11 Thread Yu Li (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17133170#comment-17133170
 ] 

Yu Li commented on FLINK-18242:
---

I agree that making `DefaultConfigurableOptionsFactory` final is not a thorough 
solution. After a second thought, we should consider completely separating 
`RocksDBOptionsFactory` and `OptionsFactory`. The only problem would be how to 
make `RocksDBStateBackend#setOptions` backward compatible (accepting an 
`OptionsFactory` and assign to a `RocksDBOptionsFactory` variable) - I think 
another {{OptionsFactory->RocksDBOptionsFactory}} adapter could make it work.

Let me prepare a PR to better Illustrate the idea.

> 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)


[jira] [Commented] (FLINK-18242) Custom OptionsFactory settings seem to have no effect on RocksDB

2020-06-11 Thread Nico Kruber (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)


[jira] [Commented] (FLINK-18242) Custom OptionsFactory settings seem to have no effect on RocksDB

2020-06-10 Thread Yu Li (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17132910#comment-17132910
 ] 

Yu Li commented on FLINK-18242:
---

Thanks for the analysis [~yunta], but you may have neglected below code path in 
`RocksDBStateBackend#setOptions`:
{code}
public void setOptions(OptionsFactory optionsFactory) {
this.rocksDbOptionsFactory = optionsFactory instanceof 
RocksDBOptionsFactory
? (RocksDBOptionsFactory) optionsFactory
: new 
RocksDBOptionsFactoryAdapter(optionsFactory);
}
{code}

While in `RocksDBOptionsFactoryAdaptor` we have below method:
{code}
public DBOptions createDBOptions(DBOptions currentOptions, 
Collection handlesToClose) {
return optionsFactory.createDBOptions(currentOptions);
}
{code}

Please notice that we also have below instructions in `RocksDBOptionsFactory` 
javadoc:
{code}
/**
 * Do not override these methods, they are only to maintain interface 
compatibility with
 * prior versions. They will be removed in one of the next versions.
 */
@Override
default DBOptions createDBOptions(DBOptions currentOptions) {
return createDBOptions(currentOptions, new ArrayList<>());
}
{code}

And since the usage mentioned in description here actually breaks the above 
rule, IMHO it's a user error.

However, we should take some actions to prevent this from happening again, 
possible ones including:
1. Make `DefaultConfigurableOptionsFactory` a final class. This may break the 
compilation of existing application code, but it's way better than having the 
problem mentioned here.
2. Write more explicit notice and suggestions in our documentation.

> 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)


[jira] [Commented] (FLINK-18242) Custom OptionsFactory settings seem to have no effect on RocksDB

2020-06-10 Thread Yun Tang (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-18242?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17132880#comment-17132880
 ] 

Yun Tang commented on FLINK-18242:
--

[~NicoK] The root cause is current 
{{[RocksDBResourceContainer|https://github.com/apache/flink/blob/88cc44afbfb5267e2674ecb735561365e735d2b0/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBResourceContainer.java#L90]}}
 would only deal with new interface 
{{DefaultConfigurableOptionsFactory#createDBOptions(DBOptions currentOptions, 
Collection handlesToClose)}}.
As you only override the deprecated interface, that's why the new options not 
take effect.

This is certainly a bug since we migrate {{DefaultConfigurableOptionsFactory}} 
to new {{RocksDBOptionsFactory}}.

> 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)