Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Status

Current state: Under Discussion Accepted

Discussion thread:  here (<- link to https://mail-archiveslists.apache.org/mod_mbox/flink-dev/)
JIRA: here (<- link to https://issues.apache.org/jira/browse/FLINK-XXXX)thread/y5owjkfxq3xs9lmpdbl6d6jmqdgbjqxo

JIRA:

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyFLINK-33581

Released: <Flink Version>

...

However, there is a significant blocker to implement the proposed solution. Currently, the non-ConfigOption objects in the StreamExecutionEnvironment, CheckpointConfig, and ExecutionConfig have already been exposed to users through the public API. This poses a challenge when trying to modify the existing implementation to accommodate the proposed solution. Therefore, this FLIP aims to deprecate these Java objects and their corresponding getter/setter interfaces, ultimately removing them in FLINK-2.0.

Please note that this FLIP does not include deprecating fields related to serialization. The deprecation work for the serialization part will be carried out in conjunction with the relevant work in the FLINK-2.0 serialization section.

Public Interfaces

Deprecate following classes, fields and methods

  • RestartStrategy:

Class

Annotation

org.apache.flink.api.common.restartstrategy.RestartStrategies

@PublicEvolving







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

Fields or Methods

Annotation

org.apache.flink.streaming.api.environment.StreamExecutionEnvironment

getRestartStrategy(

#setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration)

@Public

setRestartStrategy(RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration

org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#getRestartStrategy()

org.apache.flink.api.common.ExecutionConfig

#getRestartStrategy()

@Public

setRestartStrategy

org.apache.flink.api.common.ExecutionConfig#setRestartStrategy(RestartStrategies.

RestartStrategyConfiguration restartStrategyConfiguration

RestartStrategyConfiguration restartStrategyConfiguration)


Field

Annotation

restartStrategyConfiguration

org.apache.flink.api.common.

restartstrategy.RestartStrategies

Deprecate the entire class, includes its inner class

@PublicEvolving

ExecutionConfig#restartStrategyConfiguration

@Public

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

  • CheckpointStorage

Class

Fields or Methods
MethodAnnotation

org.apache.flink.streaming.api.environment.

CheckpointConfigsetCheckpointStorage

CheckpointConfig#setCheckpointStorage(CheckpointStorage storage)

@Public
setCheckpointStorage
org.apache.flink.streaming.api.environment.CheckpointConfig#setCheckpointStorage(String checkpointDirectory)
setCheckpointStorage
org.apache.flink.streaming.api.environment.CheckpointConfig#setCheckpointStorage(URI checkpointDirectory)
setCheckpointStorage
org.apache.flink.streaming.api.environment.CheckpointConfig#setCheckpointStorage(Path checkpointDirectory)
getCheckpointStorage
org.apache.flink.streaming.api.environment.CheckpointConfig#getCheckpointStorage()

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.

  • StateBackend

Class

Method

Fields or Methods

Annotation

org.apache.flink.streaming.api.environment.StreamExecutionEnvironment

#setStateBackend(StateBackend backend)

@Public

getStateBackend

org.apache.flink.streaming.api.environment.StreamExecutionEnvironment#getStateBackend()


Field

Annotation

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

@Public

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.

...