zhuzhurk commented on code in PR #23758:
URL: https://github.com/apache/flink/pull/23758#discussion_r1403324168


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java:
##########
@@ -676,8 +690,16 @@ public StreamExecutionEnvironment 
setStateBackend(StateBackend backend) {
     /**
      * Gets the state backend that defines how to store and checkpoint state.
      *
+     * @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 find which 
state backend is used by
+     *     state backend ConfigOption. For more details on using ConfigOption 
for state backend
+     *     configuration, please refer to the Flink documentation: <a
+     *     
href="https://nightlies.apache.org/flink/flink-docs-stable/docs/ops/state/state_backends";>state-backends</a>
      * @see #setStateBackend(StateBackend)
      */
+    @Deprecated
     @PublicEvolving
     public StateBackend getStateBackend() {
         return defaultStateBackend;

Review Comment:
   I guess we should also deprecate this field?



##########
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java:
##########
@@ -82,19 +83,26 @@ public static void main(String[] args) throws Exception {
         final ParameterTool params = ParameterTool.fromArgs(args);
 
         // create the environment to create streams and configure execution
-        final StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();
+        Configuration configuration = new Configuration();
+        final StreamExecutionEnvironment env =

Review Comment:
   The environment should be created after setting the configuration, otherwise 
the settings may not take effect because the environment may maintain its own 
configuration object and this configuration is only read in the creation 
process.



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/CheckpointConfig.java:
##########
@@ -759,9 +765,15 @@ public void setCheckpointStorage(CheckpointStorage 
storage) {
      * Configures the application to write out checkpoint snapshots to the 
configured directory. See
      * {@link FileSystemCheckpointStorage} for more details on checkpointing 
to a file system.
      *
+     * @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

Review Comment:
   I think that constructor will still be needed to create a copy of 
`CheckpointConfig`. We just need to remove the copying of  `CheckpointStorage` 
when the `checkpointStorage` field is removed in 2.0.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to