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

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

commit c2c028067b3055e41ea24c6247f01fcc6a6bd5b5
Author: xuyang <xyzhong...@163.com>
AuthorDate: Wed Jan 8 17:26:03 2025 +0800

    [FLINK-36480][table] Remove all deprecated methods in TableConfig
    
    Co-authored-by: corgy-w <corg...@qq.com>
---
 flink-python/pyflink/table/table_config.py         | 44 ----------------------
 .../org/apache/flink/table/api/TableConfig.java    | 42 +--------------------
 2 files changed, 2 insertions(+), 84 deletions(-)

diff --git a/flink-python/pyflink/table/table_config.py 
b/flink-python/pyflink/table/table_config.py
index 16e97341250..ac5eba30633 100644
--- a/flink-python/pyflink/table/table_config.py
+++ b/flink-python/pyflink/table/table_config.py
@@ -151,50 +151,6 @@ class TableConfig(object):
         else:
             raise Exception("TableConfig.max_generated_code_length should be a 
int value!")
 
-    def set_idle_state_retention_time(self,
-                                      min_time: datetime.timedelta,
-                                      max_time: datetime.timedelta):
-        """
-        Specifies a minimum and a maximum time interval for how long idle 
state, i.e., state which
-        was not updated, will be retained.
-
-        State will never be cleared until it was idle for less than the 
minimum time and will never
-        be kept if it was idle for more than the maximum time.
-
-        When new data arrives for previously cleaned-up state, the new data 
will be handled as if it
-        was the first data. This can result in previous results being 
overwritten.
-
-        Set to 0 (zero) to never clean-up the state.
-
-        Example:
-        ::
-
-            >>> table_config = TableConfig() \\
-            ...     .set_idle_state_retention_time(datetime.timedelta(days=1),
-            ...                                    datetime.timedelta(days=3))
-
-        .. note::
-
-            Cleaning up state requires additional bookkeeping which becomes 
less expensive for
-            larger differences of minTime and maxTime. The difference between 
minTime and maxTime
-            must be at least 5 minutes.
-
-            Method set_idle_state_retention_time is deprecated now. The 
suggested way to set idle
-            state retention time is 
:func:`~pyflink.table.TableConfig.set_idle_state_retention`
-            Currently, setting max_time will not work and the max_time is 
directly derived from the
-            min_time as 1.5 x min_time.
-
-        :param min_time: The minimum time interval for which idle state is 
retained. Set to
-                         0 (zero) to never clean-up the state.
-        :param max_time: The maximum time interval for which idle state is 
retained. Must be at
-                         least 5 minutes greater than minTime. Set to
-                         0 (zero) to never clean-up the state.
-        """
-        j_duration_class = get_gateway().jvm.java.time.Duration
-        j_min_time = 
j_duration_class.ofSeconds(long(round(min_time.total_seconds())))
-        j_max_time = 
j_duration_class.ofSeconds(long(round(max_time.total_seconds())))
-        self._j_table_config.setIdleStateRetentionTime(j_min_time, j_max_time)
-
     def set_idle_state_retention(self, duration: datetime.timedelta):
         """
         Specifies a retention time interval for how long idle state, i.e., 
state which
diff --git 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
index 9e2f803b3a0..2b65ef8b2c1 100644
--- 
a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
+++ 
b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
@@ -96,9 +96,8 @@ import static 
org.apache.flink.table.api.internal.TableConfigValidation.validate
 @PublicEvolving
 public final class TableConfig implements WritableConfig, ReadableConfig {
 
-    /** Please use {@link TableConfig#getDefault()} instead. */
-    @Deprecated
-    public TableConfig() {}
+    /** Please use {@link TableConfig#getDefault()} to get the default {@link 
TableConfig}. */
+    private TableConfig() {}
 
     // Note to implementers:
     // TableConfig is a ReadableConfig which is built once the 
TableEnvironment is created and
@@ -356,43 +355,6 @@ public final class TableConfig implements WritableConfig, 
ReadableConfig {
                 TableConfigOptions.MAX_LENGTH_GENERATED_CODE, 
maxGeneratedCodeLength);
     }
 
-    /**
-     * Specifies a minimum and a maximum time interval for how long idle 
state, i.e., state which
-     * was not updated, will be retained. State will never be cleared until it 
was idle for less
-     * than the minimum time and will never be kept if it was idle for more 
than the maximum time.
-     *
-     * <p>When new data arrives for previously cleaned-up state, the new data 
will be handled as if
-     * it was the first data. This can result in previous results being 
overwritten.
-     *
-     * <p>Set to 0 (zero) to never clean-up the state.
-     *
-     * <p>NOTE: Cleaning up state requires additional bookkeeping which 
becomes less expensive for
-     * larger differences of minTime and maxTime. The difference between 
minTime and maxTime must be
-     * at least 5 minutes.
-     *
-     * <p>NOTE: Currently maxTime will be ignored and it will automatically 
derived from minTime as
-     * 1.5 x minTime.
-     *
-     * @param minTime The minimum time interval for which idle state is 
retained. Set to 0 (zero) to
-     *     never clean-up the state.
-     * @param maxTime The maximum time interval for which idle state is 
retained. Must be at least 5
-     *     minutes greater than minTime. Set to 0 (zero) to never clean-up the 
state.
-     * @deprecated use {@link #setIdleStateRetention(Duration)} instead.
-     */
-    @Deprecated
-    public void setIdleStateRetentionTime(Duration minTime, Duration maxTime) {
-        if (maxTime.minus(minTime).toMillis() < 300000
-                && !(maxTime.toMillis() == 0 && minTime.toMillis() == 0)) {
-            throw new IllegalArgumentException(
-                    "Difference between minTime: "
-                            + minTime
-                            + " and maxTime: "
-                            + maxTime
-                            + " should be at least 5 minutes.");
-        }
-        setIdleStateRetention(minTime);
-    }
-
     /**
      * Specifies a retention time interval for how long idle state, i.e., 
state which was not
      * updated, will be retained. State will never be cleared until it was 
idle for less than the

Reply via email to