This is an automated email from the ASF dual-hosted git repository.

zhuzh pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0e0099b4eb1285929fec02326f661cba899eedcd
Author: JunRuiLee <jrlee....@gmail.com>
AuthorDate: Mon Nov 20 16:14:56 2023 +0800

    [FLINK-33581][core] Deprecate complex Java objects and their corresponding 
getter/setter methods related to restart strategies.
    
    This closes #23758.
---
 .../5b9eed8a-5fb6-4373-98ac-3be2a71941b8           |  4 ----
 .../apache/flink/api/common/ExecutionConfig.java   | 24 +++++++++++++++++++++-
 .../common/restartstrategy/RestartStrategies.java  | 18 ++++++++++++++++
 .../environment/StreamExecutionEnvironment.java    | 14 +++++++++++++
 4 files changed, 55 insertions(+), 5 deletions(-)

diff --git 
a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
 
b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
index cad03129799..7510d7d0190 100644
--- 
a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
+++ 
b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
@@ -1,5 +1,3 @@
-org.apache.flink.api.common.ExecutionConfig.getRestartStrategy(): Returned 
leaf type 
org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration
 does not satisfy: reside outside of package 'org.apache.flink..' or reside in 
any package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.common.ExecutionConfig.setRestartStrategy(org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration):
 Argument leaf type 
org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration
 does not satisfy: reside outside of package 'org.apache.flink..' or reside in 
any package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 
org.apache.flink.api.common.eventtime.TimestampAssignerSupplier.createTimestampAssigner(org.apache.flink.api.common.eventtime.TimestampAssignerSupplier$Context):
 Argument leaf type 
org.apache.flink.api.common.eventtime.TimestampAssignerSupplier$Context does 
not satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 
org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.createWatermarkGenerator(org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier$Context):
 Argument leaf type 
org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier$Context does 
not satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 org.apache.flink.api.common.operators.Operator.getMinResources(): Returned 
leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: 
reside outside of package 'org.apache.flink..' or reside in any package 
['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or 
annotated with @Deprecated
@@ -185,8 +183,6 @@ 
org.apache.flink.streaming.api.datastream.BroadcastStream.getTransformation(): R
 org.apache.flink.streaming.api.datastream.DataStream.getMinResources(): 
Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not 
satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 org.apache.flink.streaming.api.datastream.DataStream.getPreferredResources(): 
Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not 
satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 
org.apache.flink.streaming.api.datastream.DataStream.transform(java.lang.String,
 org.apache.flink.api.common.typeinfo.TypeInformation, 
org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory): 
Argument leaf type 
org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory does not 
satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
-org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.getRestartStrategy():
 Returned leaf type 
org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration
 does not satisfy: reside outside of package 'org.apache.flink..' or reside in 
any package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
-org.apache.flink.streaming.api.environment.StreamExecutionEnvironment.setRestartStrategy(org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration):
 Argument leaf type 
org.apache.flink.api.common.restartstrategy.RestartStrategies$RestartStrategyConfiguration
 does not satisfy: reside outside of package 'org.apache.flink..' or reside in 
any package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 org.apache.flink.streaming.api.functions.KeyedProcessFunction.onTimer(long, 
org.apache.flink.streaming.api.functions.KeyedProcessFunction$OnTimerContext, 
org.apache.flink.util.Collector): Argument leaf type 
org.apache.flink.streaming.api.functions.KeyedProcessFunction$OnTimerContext 
does not satisfy: reside outside of package 'org.apache.flink..' or reside in 
any package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 
org.apache.flink.streaming.api.functions.KeyedProcessFunction.processElement(java.lang.Object,
 org.apache.flink.streaming.api.functions.KeyedProcessFunction$Context, 
org.apache.flink.util.Collector): Argument leaf type 
org.apache.flink.streaming.api.functions.KeyedProcessFunction$Context does not 
satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
 org.apache.flink.streaming.api.functions.ProcessFunction.onTimer(long, 
org.apache.flink.streaming.api.functions.ProcessFunction$OnTimerContext, 
org.apache.flink.util.Collector): Argument leaf type 
org.apache.flink.streaming.api.functions.ProcessFunction$OnTimerContext does 
not satisfy: reside outside of package 'org.apache.flink..' or reside in any 
package ['..shaded..'] or annotated with @Public or annotated with 
@PublicEvolving or annotated with @Deprecated
diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java 
b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index 22aee331513..3b0c73eba00 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -149,6 +149,15 @@ public class ExecutionConfig implements Serializable, 
Archiveable<ArchivedExecut
      */
     @Deprecated private long executionRetryDelay = DEFAULT_RESTART_DELAY;
 
+    /**
+     * @deprecated The field is marked as deprecated because starting from 
Flink 1.19, the usage of
+     *     all complex Java objects related to configuration, including their 
getter and setter
+     *     methods, should be replaced by ConfigOption. In a future major 
version of Flink, this
+     *     method will be removed entirely. It is recommended to switch to 
using the ConfigOptions
+     *     provided by {@link 
org.apache.flink.configuration.RestartStrategyOptions} for configuring
+     *     restart strategies.
+     */
+    @Deprecated
     private RestartStrategies.RestartStrategyConfiguration 
restartStrategyConfiguration =
             new RestartStrategies.FallbackRestartStrategyConfiguration();
 
@@ -434,8 +443,15 @@ public class ExecutionConfig implements Serializable, 
Archiveable<ArchivedExecut
      *         1000 // delay between retries));
      * }</pre>
      *
+     * @deprecated The method is marked as deprecated because starting from 
Flink 1.19, the usage of
+     *     all complex Java objects related to configuration, including their 
getter and setter
+     *     methods, should be replaced by ConfigOption. In a future major 
version of Flink, this
+     *     method will be removed entirely. It is recommended to switch to 
using the ConfigOptions
+     *     provided by {@link 
org.apache.flink.configuration.RestartStrategyOptions} for configuring
+     *     restart strategies.
      * @param restartStrategyConfiguration Configuration defining the restart 
strategy to use
      */
+    @Deprecated
     @PublicEvolving
     public void setRestartStrategy(
             RestartStrategies.RestartStrategyConfiguration 
restartStrategyConfiguration) {
@@ -446,10 +462,16 @@ public class ExecutionConfig implements Serializable, 
Archiveable<ArchivedExecut
     /**
      * Returns the restart strategy which has been set for the current job.
      *
+     * @deprecated The method is marked as deprecated because starting from 
Flink 1.19, the usage of
+     *     all complex Java objects related to configuration, including their 
getter and setter
+     *     methods, should be replaced by ConfigOption. In a future major 
version of Flink, this
+     *     method will be removed entirely. It is recommended to switch to 
using the ConfigOptions
+     *     provided by {@link 
org.apache.flink.configuration.RestartStrategyOptions} for configuring
+     *     restart strategies.
      * @return The specified restart configuration
      */
+    @Deprecated
     @PublicEvolving
-    @SuppressWarnings("deprecation")
     public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() 
{
         if (restartStrategyConfiguration
                 instanceof 
RestartStrategies.FallbackRestartStrategyConfiguration) {
diff --git 
a/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java
 
b/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java
index 54a9c212cfb..10363d77f65 100644
--- 
a/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java
+++ 
b/flink-core/src/main/java/org/apache/flink/api/common/restartstrategy/RestartStrategies.java
@@ -35,7 +35,25 @@ import java.util.concurrent.TimeUnit;
  *
  * <p>The RestartStrategyConfigurations are used to decouple the core module 
from the runtime
  * module.
+ *
+ * @deprecated The {@link RestartStrategies} class is marked as deprecated 
because starting from
+ *     Flink 1.19, all complex Java objects related to configuration should be 
replaced by
+ *     ConfigOption. In a future major version of Flink, this class will be 
removed entirely. It is
+ *     recommended to switch to using the ConfigOptions provided by {@link
+ *     org.apache.flink.configuration.RestartStrategyOptions} for configuring 
restart strategies
+ *     like the following code snippet:
+ *     <pre>{@code
+ * Configuration config = new Configuration();
+ * config.set(RestartStrategyOptions.RESTART_STRATEGY, "fixed-delay");
+ * config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_ATTEMPTS, 3);
+ * config.set(RestartStrategyOptions.RESTART_STRATEGY_FIXED_DELAY_DELAY, 
Duration.ofMinutes(1));
+ * StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment(config);
+ * }</pre>
+ *     For more details on using ConfigOption for restart strategies, please 
refer to the Flink
+ *     documentation: <a
+ *     
href="https://nightlies.apache.org/flink/flink-docs-stable/docs/ops/state/task_failure_recovery/#restart-strategies";>restart-strategies</a>
  */
+@Deprecated
 @PublicEvolving
 public class RestartStrategies {
 
diff --git 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
index 4ecb9763f49..52b5e30034f 100644
--- 
a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
+++ 
b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
@@ -788,8 +788,15 @@ public class StreamExecutionEnvironment implements 
AutoCloseable {
      * Sets the restart strategy configuration. The configuration specifies 
which restart strategy
      * will be used for the execution graph in case of a restart.
      *
+     * @deprecated The method is marked as deprecated because starting from 
Flink 1.19, the usage of
+     *     all complex Java objects related to configuration, including their 
getter and setter
+     *     methods, should be replaced by ConfigOption. In a future major 
version of Flink, this
+     *     method will be removed entirely. It is recommended to switch to 
using the ConfigOptions
+     *     provided by {@link 
org.apache.flink.configuration.RestartStrategyOptions} for configuring
+     *     restart strategies.
      * @param restartStrategyConfiguration Restart strategy configuration to 
be set
      */
+    @Deprecated
     @PublicEvolving
     public void setRestartStrategy(
             RestartStrategies.RestartStrategyConfiguration 
restartStrategyConfiguration) {
@@ -799,8 +806,15 @@ public class StreamExecutionEnvironment implements 
AutoCloseable {
     /**
      * Returns the specified restart strategy configuration.
      *
+     * @deprecated The method is marked as deprecated because starting from 
Flink 1.19, the usage of
+     *     all complex Java objects related to configuration, including their 
getter and setter
+     *     methods, should be replaced by ConfigOption. In a future major 
version of Flink, this
+     *     method will be removed entirely. It is recommended to switch to 
using the ConfigOptions
+     *     provided by {@link 
org.apache.flink.configuration.RestartStrategyOptions} for configuring
+     *     restart strategies.
      * @return The restart strategy configuration to be used
      */
+    @Deprecated
     @PublicEvolving
     public RestartStrategies.RestartStrategyConfiguration getRestartStrategy() 
{
         return config.getRestartStrategy();

Reply via email to