You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Junrui Li (Jira)" <ji...@apache.org> on 2024/01/31 02:26:00 UTC

[jira] [Updated] (FLINK-33581) FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects

     [ https://issues.apache.org/jira/browse/FLINK-33581?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Junrui Li updated FLINK-33581:
------------------------------
    Release Note: 
The non-ConfigOption objects in the StreamExecutionEnvironment, CheckpointConfig, and ExecutionConfig and their corresponding getter/setter interfaces is now be deprecated in FLINK-1.19. And these objects and methods is planned to be removed in Flink-2.0. Detailed information regarding the deprecations is as follows:

1. RestartStrategy:
Class:
org.apache.flink.api.common.restartstrategy.RestartStrategies
org.apache.flink.api.common.restartstrategy.RestartStrategies.RestartStrategyConfiguration
org.apache.flink.api.common.restartstrategy.RestartStrategies.FixedDelayRestartStrategyConfiguration
org.apache.flink.api.common.restartstrategy.RestartStrategies.ExponentialDelayRestartStrategyConfiguration
org.apache.flink.api.common.restartstrategy.RestartStrategies.FailureRateRestartStrategyConfiguration
org.apache.flink.api.common.restartstrategy.RestartStrategies.FallbackRestartStrategyConfiguration

Method:
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration)
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#getRestartStrategy()
org.apache.flink.api.common.ExecutionConfig#getRestartStrategy()
org.apache.flink.api.common.ExecutionConfig#setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration)
pyflink.common.execution_config.ExecutionConfig.set_restart_strategy(self, restart_strategy_configuration: RestartStrategyConfiguration)
pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment.set_restart_strategy(self, restart_strategy_configuration: RestartStrategyConfiguration)

Field:
org.apache.flink.api.common.ExecutionConfig#restartStrategyConfiguration

Suggested alternative: 
Users can configure the RestartStrategyOptions related ConfigOptions, such as "restart-strategy.type", in the configuration, instead of passing a RestartStrategyConfiguration object.

2.CheckpointStorage:
Method:
org.apache.flink.streaming.api.environment.CheckpointConfig#setCheckpointStorage(CheckpointStorage storage)
org.apache.flink.streaming.api.environment.CheckpointConfig#setCheckpointStorage(String checkpointDirectory)
org.apache.flink.streaming.api.environment.CheckpointConfig#setCheckpointStorage(URI checkpointDirectory)
org.apache.flink.streaming.api.environment.CheckpointConfig#setCheckpointStorage(Path checkpointDirectory)
org.apache.flink.streaming.api.environment.CheckpointConfig#getCheckpointStorage()
pyflink.datastream.checkpoint_config.CheckpointConfig.set_checkpoint_storage(self, storage: CheckpointStorage)
pyflink.datastream.checkpoint_config.CheckpointConfig.set_checkpoint_storage_dir(self, checkpoint_path: str)
pyflink.datastream.checkpoint_config.CheckpointConfig.get_checkpoint_storage(self)

Suggested alternative: 
Users can configure "state.checkpoint-storage" in the configuration as the fully qualified name of the checkpoint storage or use some FLINK-provided checkpoint storage shortcut names such as "jobmanager" and "filesystem", and provide the necessary configuration options for building that storage, instead of passing a CheckpointStorage object.

3.StateBackend:
Method:
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#setStateBackend(StateBackend backend)
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#getStateBackend()
pyflink.datastream.stream_execution_environment.StreamExecutionEnvironment.set_state_backend(self, state_backend: StateBackend)

Field:
org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#defaultStateBackend

Suggested alternative: 
Users can configure "state.backend.type" in the configuration as the fully qualified name of the state backend or use some FLINK-provided state backend shortcut names such as "hashmap" and "rocksdb", and provide the necessary configuration options for building that StateBackend, instead of passing a StateBackend object.


More details can be found at https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=278464992.

> FLIP-381: Deprecate configuration getters/setters that return/set complex Java objects
> --------------------------------------------------------------------------------------
>
>                 Key: FLINK-33581
>                 URL: https://issues.apache.org/jira/browse/FLINK-33581
>             Project: Flink
>          Issue Type: Technical Debt
>          Components: API / DataStream
>            Reporter: Junrui Li
>            Assignee: Junrui Li
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.19.0
>
>         Attachments: image-2023-11-30-17-59-42-650.png
>
>
> Deprecate the non-ConfigOption objects in the StreamExecutionEnvironment, CheckpointConfig, and ExecutionConfig, and ultimately removing them in FLINK 2.0



--
This message was sent by Atlassian Jira
(v8.20.10#820010)