You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2022/03/22 19:13:48 UTC

[flink] branch release-1.15 updated (41ee1d2 -> 65a3b08)

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

twalthr pushed a change to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git.


    from 41ee1d2  [FLINK-26536][python] Fix RemoteKeyedStateBackend#merge_namespaces to handle properly for the timerservice
     new 0ba337c  [hotfix][connector-hive] Avoid serializing TableConfig
     new 5bd6aef  [FLINK-26709][table] Replace TableConfig.getConfiguration.set()
     new a6daa22  [FLINK-26709][table] Replace TableConfig.getConfiguration.get/getOptional()
     new 885d4bb  [FLINK-26709][table] Replace TableConfig.getConfiguration
     new b7fb464  [FLINK-26777][table-planner] Remove `PlannerConfig` from `PlannerBase`
     new 607c3a7  [hotfix][table-planner][tests] Use EnvironmentSettings to pass configuration
     new 30ce8e7  [hotfix][table-planner] Remove ExecNodeConfig#getLocalTimeZone
     new 65a3b08  [hotfix][table-planner] Remove ExecNodeConfig#getMaxIdleStateRetentionTime()

The 8 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../table/ElasticsearchDynamicSinkBaseITCase.java  |   6 +-
 .../hive/ContinuousHiveSplitEnumerator.java        |  13 +-
 .../connectors/hive/HiveLookupTableSource.java     |   2 +-
 .../apache/flink/connectors/hive/HiveSource.java   |  11 +-
 .../flink/connectors/hive/HiveSourceBuilder.java   |  18 ++-
 .../connectors/hive/HiveSourceFileEnumerator.java  |  29 ++--
 .../flink/connectors/hive/HiveTableSink.java       |  33 +++-
 .../flink/connectors/hive/HiveTableSource.java     |   4 +-
 .../hive/read/HiveInputFormatPartitionReader.java  |   9 +-
 .../connectors/hive/read/HiveTableInputFormat.java |  20 +--
 .../hive/HiveDynamicTableFactoryTest.java          |   4 +-
 .../connectors/hive/HiveLookupJoinITCase.java      |   2 +-
 .../flink/connectors/hive/HiveRunnerITCase.java    |   4 +-
 .../connectors/hive/HiveSinkCompactionITCase.java  |   4 +-
 .../flink/connectors/hive/HiveTableSinkITCase.java |   8 +-
 .../connectors/hive/HiveTableSourceITCase.java     |  31 +---
 .../connectors/hive/PartitionMonitorTest.java      |   3 +-
 .../hive/TableEnvHiveConnectorITCase.java          |   8 +-
 .../read/HiveInputFormatPartitionReaderITCase.java |   4 +-
 .../table/catalog/hive/HiveCatalogITCase.java      |  14 +-
 .../flink/table/catalog/hive/HiveTestUtils.java    |   8 +-
 .../connector/jdbc/catalog/MySqlCatalogITCase.java |   4 +-
 .../jdbc/catalog/PostgresCatalogITCase.java        |   4 +-
 .../kafka/table/KafkaChangelogTableITCase.java     |  37 +++--
 .../connectors/kafka/table/KafkaTableITCase.java   |  11 +-
 .../flink/python/tests/BatchPythonUdfSqlJob.java   |   2 +-
 .../apache/flink/table/tpcds/TpcdsTestProgram.java |  12 +-
 .../flink/client/python/PythonFunctionFactory.java |   7 +-
 .../flink/table/runtime/arrow/ArrowUtils.java      |   2 +-
 .../client/python/PythonFunctionFactoryTest.java   |   5 +-
 .../PythonScalarFunctionOperatorTestBase.java      |   5 +-
 .../flink/table/client/cli/CliResultViewTest.java  |   4 +-
 .../flink/table/client/cli/TestingExecutor.java    |   3 +-
 .../client/gateway/context/SessionContextTest.java |  39 +++--
 .../AbstractStreamTableEnvironmentImpl.java        |   3 +-
 .../org/apache/flink/table/api/TableResult.java    |   4 +-
 .../flink/table/api/internal/CompiledPlanImpl.java |   5 +-
 .../table/api/internal/TableEnvironmentImpl.java   |  15 +-
 .../flink/table/catalog/FunctionCatalog.java       |   6 -
 .../expressions/resolver/ExpressionResolver.java   |   2 +-
 .../flink/table/utils/TableEnvironmentMock.java    |   3 +-
 .../internal/StreamTableEnvironmentImpl.scala      |   2 +-
 .../planner/connectors/DynamicSourceUtils.java     |   4 +-
 .../table/planner/delegation/PlannerConfig.java    |  56 -------
 .../planner/plan/QueryOperationConverter.java      |   4 +-
 .../abilities/source/WatermarkPushDownSpec.java    |   5 +-
 .../planner/plan/nodes/exec/ExecNodeBase.java      |   4 +-
 .../planner/plan/nodes/exec/ExecNodeConfig.java    |  59 ++------
 .../exec/common/CommonExecWindowTableFunction.java |   4 +-
 .../exec/processor/DeadlockBreakupProcessor.java   |   2 +-
 .../MultipleInputNodeCreationProcessor.java        |   2 +-
 .../plan/nodes/exec/serde/SerdeContext.java        |   6 +-
 .../plan/nodes/exec/spec/DynamicTableSinkSpec.java |   2 +-
 .../nodes/exec/spec/DynamicTableSourceSpec.java    |   2 +-
 .../plan/nodes/exec/spec/DynamicTableSpecBase.java |   5 +-
 .../stream/StreamExecGlobalWindowAggregate.java    |   4 +-
 .../stream/StreamExecGroupWindowAggregate.java     |   3 +-
 .../stream/StreamExecLocalWindowAggregate.java     |   4 +-
 .../nodes/exec/stream/StreamExecOverAggregate.java |  11 +-
 .../stream/StreamExecPythonGroupAggregate.java     |   3 +-
 .../StreamExecPythonGroupTableAggregate.java       |   3 +-
 .../StreamExecPythonGroupWindowAggregate.java      |   3 +-
 .../exec/stream/StreamExecPythonOverAggregate.java |   3 +-
 .../nodes/exec/stream/StreamExecTemporalJoin.java  |   3 +-
 .../exec/stream/StreamExecWindowAggregate.java     |   4 +-
 .../exec/stream/StreamExecWindowDeduplicate.java   |   4 +-
 .../nodes/exec/stream/StreamExecWindowJoin.java    |   4 +-
 .../nodes/exec/stream/StreamExecWindowRank.java    |   4 +-
 .../logical/PushFilterIntoSourceScanRuleBase.java  |   6 +-
 .../PushWatermarkIntoTableSourceScanRuleBase.java  |   4 +-
 .../batch/PushLocalAggIntoScanRuleBase.java        |   6 +-
 .../planner/plan/schema/CatalogSourceTable.java    |   9 +-
 .../utils/DummyStreamExecutionEnvironment.java     |   6 +
 .../flink/table/planner/utils/ShortcutUtils.java   |   2 +-
 .../table/planner/utils/TableConfigUtils.java      |  49 +++++-
 .../table/planner/delegation/BatchPlanner.scala    |   5 +-
 .../table/planner/delegation/PlannerBase.scala     |  35 ++---
 .../table/planner/delegation/StreamPlanner.scala   |   3 +-
 .../plan/metadata/FlinkRelMdColumnNullCount.scala  |   3 +-
 .../plan/metadata/FlinkRelMdDistinctRowCount.scala |   3 +-
 .../plan/metadata/FlinkRelMdDistribution.scala     |   3 +-
 .../planner/plan/metadata/FlinkRelMdRowCount.scala |   3 +-
 .../plan/metadata/SelectivityEstimator.scala       |   3 +-
 .../plan/nodes/logical/FlinkLogicalSort.scala      |   4 +-
 .../batch/BatchPhysicalHashAggregate.scala         |   2 +-
 .../batch/BatchPhysicalOverAggregateBase.scala     |   2 +-
 .../batch/BatchPhysicalPythonGroupAggregate.scala  |   2 +-
 .../nodes/physical/batch/BatchPhysicalRank.scala   |   2 +-
 .../batch/BatchPhysicalSortAggregate.scala         |   2 +-
 .../BatchCommonSubGraphBasedOptimizer.scala        |   4 +-
 .../StreamCommonSubGraphBasedOptimizer.scala       |  10 +-
 .../plan/optimize/program/FlinkBatchProgram.scala  |   6 +-
 .../FlinkChangelogModeInferenceProgram.scala       |   2 +-
 .../plan/optimize/program/FlinkStreamProgram.scala |   6 +-
 .../table/planner/plan/reuse/SubplanReuser.scala   |   5 +-
 .../rules/logical/JoinDeriveNullFilterRule.scala   |   2 +-
 .../PushFilterIntoLegacyTableSourceScanRule.scala  |   2 +-
 .../plan/rules/logical/SplitAggregateRule.scala    |   4 +-
 .../physical/batch/BatchPhysicalHashJoinRule.scala |   4 +-
 .../physical/batch/BatchPhysicalJoinRuleBase.scala |   2 +-
 .../batch/BatchPhysicalSortMergeJoinRule.scala     |   4 +-
 .../physical/batch/BatchPhysicalSortRule.scala     |   2 +-
 .../physical/stream/IncrementalAggregateRule.scala |   2 +-
 .../stream/MiniBatchIntervalInferRule.scala        |   3 +-
 .../stream/TwoStageOptimizedAggregateRule.scala    |   3 +-
 .../plan/schema/LegacyCatalogSourceTable.scala     |   4 +-
 .../table/planner/plan/utils/FlinkRelOptUtil.scala |   3 +-
 .../planner/plan/utils/IntervalJoinUtil.scala      |   3 +-
 .../table/planner/plan/utils/PartitionPruner.scala |   7 +-
 .../flink/table/planner/plan/utils/RankUtil.scala  |   3 +-
 .../table/planner/plan/utils/WindowUtil.scala      |   5 +-
 .../BuiltInAggregateFunctionTestBase.java          |   4 +-
 .../nodes/exec/serde/LogicalTypeJsonSerdeTest.java |  16 +-
 .../plan/nodes/exec/serde/LookupKeySerdeTest.java  |   2 +-
 .../PushLocalAggIntoTableSourceScanRuleTest.java   |   3 +-
 .../flink/table/api/TableEnvironmentTest.scala     |  11 +-
 .../planner/calcite/CalciteConfigBuilderTest.scala |   4 +-
 .../plan/batch/sql/DeadlockBreakupTest.scala       | 110 +++++++-------
 .../planner/plan/batch/sql/LegacySinkTest.scala    |   5 +-
 .../plan/batch/sql/MultipleInputCreationTest.scala |  56 +++----
 .../plan/batch/sql/RemoveCollationTest.scala       |  40 ++---
 .../planner/plan/batch/sql/RemoveShuffleTest.scala | 168 ++++++++++-----------
 .../planner/plan/batch/sql/SetOperatorsTest.scala  |   2 +-
 .../planner/plan/batch/sql/SortLimitTest.scala     |  24 +--
 .../table/planner/plan/batch/sql/SortTest.scala    |  34 ++---
 .../planner/plan/batch/sql/SubplanReuseTest.scala  |  68 ++++-----
 .../planner/plan/batch/sql/TableSinkTest.scala     |   5 +-
 .../planner/plan/batch/sql/TableSourceTest.scala   |   5 +-
 .../plan/batch/sql/agg/GroupWindowTest.scala       |   2 +-
 .../plan/batch/sql/agg/HashAggregateTest.scala     |   4 +-
 .../plan/batch/sql/agg/SortAggregateTest.scala     |   4 +-
 .../batch/sql/join/BroadcastHashJoinTest.scala     |   7 +-
 .../sql/join/BroadcastHashSemiAntiJoinTest.scala   |   6 +-
 .../plan/batch/sql/join/LookupJoinTest.scala       |   8 +-
 .../plan/batch/sql/join/NestedLoopJoinTest.scala   |   2 +-
 .../sql/join/NestedLoopSemiAntiJoinTest.scala      |   2 +-
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |   2 +-
 .../sql/join/ShuffledHashSemiAntiJoinTest.scala    |   2 +-
 .../plan/batch/sql/join/SortMergeJoinTest.scala    |   2 +-
 .../batch/sql/join/SortMergeSemiAntiJoinTest.scala |   2 +-
 .../planner/plan/common/JoinReorderTestBase.scala  |   8 +-
 .../planner/plan/common/TableFactoryTest.scala     |   2 +-
 .../table/planner/plan/hint/OptionsHintTest.scala  |   4 +-
 .../plan/metadata/FlinkRelMdHandlerTestBase.scala  |   6 +-
 .../logical/JoinDeriveNullFilterRuleTest.scala     |   4 +-
 .../rules/logical/SplitAggregateRuleTest.scala     |  16 +-
 .../batch/EnforceLocalHashAggRuleTest.scala        |   6 +-
 .../batch/EnforceLocalSortAggRuleTest.scala        |   6 +-
 .../RemoveRedundantLocalHashAggRuleTest.scala      |  18 +--
 .../stream/ChangelogModeInferenceTest.scala        |   8 +-
 ...xpandWindowTableFunctionTransposeRuleTest.scala |   4 +-
 .../plan/stream/sql/DagOptimizationTest.scala      |  55 +++----
 .../planner/plan/stream/sql/DeduplicateTest.scala  |  14 +-
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  42 +++---
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  18 +--
 .../table/planner/plan/stream/sql/RankTest.scala   |   5 +-
 .../planner/plan/stream/sql/SubplanReuseTest.scala |  34 ++---
 .../planner/plan/stream/sql/TableScanTest.scala    |  12 +-
 .../planner/plan/stream/sql/TableSinkTest.scala    |  23 ++-
 .../plan/stream/sql/agg/AggregateTest.scala        |  24 +--
 .../stream/sql/agg/DistinctAggregateTest.scala     |  11 +-
 .../plan/stream/sql/agg/GroupWindowTest.scala      |  16 +-
 .../stream/sql/agg/IncrementalAggregateTest.scala  |   4 +-
 .../stream/sql/agg/TwoStageAggregateTest.scala     |   2 +-
 .../plan/stream/sql/agg/WindowAggregateTest.scala  |  38 ++---
 .../sql/agg/AggregateJoinTransposeITCase.scala     |   2 +-
 .../runtime/stream/sql/GroupWindowITCase.scala     |   5 +-
 .../utils/StreamingWithMiniBatchTestBase.scala     |   6 +-
 .../flink/table/planner/utils/TableTestBase.scala  |  11 +-
 .../GeneratedWatermarkGeneratorSupplier.java       |   9 +-
 170 files changed, 881 insertions(+), 990 deletions(-)
 delete mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerConfig.java

[flink] 03/08: [FLINK-26709][table] Replace TableConfig.getConfiguration.get/getOptional()

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit a6daa22d63df261b228c22f54564c17c243e57d8
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 14:32:23 2022 +0200

    [FLINK-26709][table] Replace TableConfig.getConfiguration.get/getOptional()
    
    Replace `TableConfig.getConfiguration().get(<option>)/getOptional(<option>)`
     with `TableConfig.get(<option>)/getOptional(<option>)` since `TableConfig`
    is now a `ReadableConfig` and the `get/getOptional` give a full view,
    including the `rootConfiguration`, which makes all the options coming
    from the environment (flink-conf.yaml, CLI params) available.
    
    (cherry picked from commit 0b4b5357f7a793b01948b3195efa65582c06e1b6)
---
 .../flink/table/runtime/arrow/ArrowUtils.java      |  2 +-
 .../client/gateway/context/SessionContextTest.java | 30 +++++++++++-----------
 .../flink/table/api/internal/CompiledPlanImpl.java |  5 +---
 .../table/api/internal/TableEnvironmentImpl.java   |  4 +--
 .../planner/connectors/DynamicSourceUtils.java     |  4 +--
 .../logical/PushFilterIntoSourceScanRuleBase.java  |  6 ++---
 .../PushWatermarkIntoTableSourceScanRuleBase.java  |  4 +--
 .../batch/PushLocalAggIntoScanRuleBase.java        |  6 ++---
 .../planner/plan/schema/CatalogSourceTable.java    |  3 +--
 .../table/planner/utils/TableConfigUtils.java      |  5 ++--
 .../table/planner/delegation/BatchPlanner.scala    |  5 ++--
 .../table/planner/delegation/PlannerBase.scala     |  2 +-
 .../plan/metadata/FlinkRelMdColumnNullCount.scala  |  3 +--
 .../plan/metadata/FlinkRelMdDistinctRowCount.scala |  3 +--
 .../plan/metadata/FlinkRelMdDistribution.scala     |  3 +--
 .../planner/plan/metadata/FlinkRelMdRowCount.scala |  3 +--
 .../plan/metadata/SelectivityEstimator.scala       |  3 +--
 .../plan/nodes/logical/FlinkLogicalSort.scala      |  4 +--
 .../batch/BatchPhysicalHashAggregate.scala         |  2 +-
 .../batch/BatchPhysicalOverAggregateBase.scala     |  2 +-
 .../batch/BatchPhysicalPythonGroupAggregate.scala  |  2 +-
 .../nodes/physical/batch/BatchPhysicalRank.scala   |  2 +-
 .../batch/BatchPhysicalSortAggregate.scala         |  2 +-
 .../plan/optimize/program/FlinkBatchProgram.scala  |  6 ++---
 .../FlinkChangelogModeInferenceProgram.scala       |  2 +-
 .../plan/optimize/program/FlinkStreamProgram.scala |  6 ++---
 .../table/planner/plan/reuse/SubplanReuser.scala   |  5 ++--
 .../rules/logical/JoinDeriveNullFilterRule.scala   |  2 +-
 .../PushFilterIntoLegacyTableSourceScanRule.scala  |  2 +-
 .../plan/rules/logical/SplitAggregateRule.scala    |  4 +--
 .../physical/batch/BatchPhysicalHashJoinRule.scala |  4 +--
 .../physical/batch/BatchPhysicalJoinRuleBase.scala |  2 +-
 .../batch/BatchPhysicalSortMergeJoinRule.scala     |  4 +--
 .../physical/batch/BatchPhysicalSortRule.scala     |  2 +-
 .../physical/stream/IncrementalAggregateRule.scala |  2 +-
 .../stream/MiniBatchIntervalInferRule.scala        |  3 +--
 .../stream/TwoStageOptimizedAggregateRule.scala    |  3 +--
 .../plan/schema/LegacyCatalogSourceTable.scala     |  4 +--
 .../table/planner/plan/utils/FlinkRelOptUtil.scala |  3 +--
 .../planner/plan/utils/IntervalJoinUtil.scala      |  3 +--
 .../flink/table/planner/plan/utils/RankUtil.scala  |  3 +--
 .../table/planner/plan/utils/WindowUtil.scala      |  5 ++--
 .../planner/calcite/CalciteConfigBuilderTest.scala |  4 +--
 .../plan/batch/sql/join/LookupJoinTest.scala       |  4 +--
 .../runtime/stream/sql/GroupWindowITCase.scala     |  5 ++--
 45 files changed, 78 insertions(+), 105 deletions(-)

diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java
index 46448bf..5e8b09f 100644
--- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java
+++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/ArrowUtils.java
@@ -567,7 +567,7 @@ public final class ArrowUtils {
         TableEnvironment tableEnv = ((TableImpl) table).getTableEnvironment();
         if (tableEnv instanceof TableEnvironmentImpl) {
             final RuntimeExecutionMode mode =
-                    tableEnv.getConfig().getConfiguration().get(ExecutionOptions.RUNTIME_MODE);
+                    tableEnv.getConfig().get(ExecutionOptions.RUNTIME_MODE);
             if (mode == RuntimeExecutionMode.AUTOMATIC) {
                 throw new RuntimeException(
                         String.format("Runtime execution mode '%s' is not supported yet.", mode));
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
index 36dadee..2d93850 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
@@ -83,19 +83,19 @@ public class SessionContextTest {
         sessionContext.set(NAME.key(), "test");
         // runtime config from flink-conf
         sessionContext.set(OBJECT_REUSE.key(), "false");
-        assertEquals("hive", getConfiguration().getString(TABLE_SQL_DIALECT));
-        assertEquals(128, getConfiguration().getInteger(MAX_PARALLELISM));
-        assertEquals("test", getConfiguration().getString(NAME));
-        assertFalse(getConfiguration().getBoolean(OBJECT_REUSE));
+        assertEquals("hive", getConfiguration().get(TABLE_SQL_DIALECT));
+        assertEquals(128, (int) getConfiguration().get(MAX_PARALLELISM));
+        assertEquals("test", getConfiguration().get(NAME));
+        assertFalse(getConfiguration().get(OBJECT_REUSE));
 
         sessionContext.reset();
-        assertEquals("default", getConfiguration().getString(TABLE_SQL_DIALECT));
+        assertEquals("default", getConfiguration().get(TABLE_SQL_DIALECT));
         assertNull(getConfiguration().get(NAME));
         // The value of MAX_PARALLELISM in DEFAULTS_ENVIRONMENT_FILE is 16
-        assertEquals(16, getConfiguration().getInteger(MAX_PARALLELISM));
-        assertNull(getConfiguration().getString(NAME, null));
+        assertEquals(16, (int) getConfiguration().get(MAX_PARALLELISM));
+        assertNull(getConfiguration().get(NAME));
         // The value of OBJECT_REUSE in origin configuration is true
-        assertTrue(getConfiguration().getBoolean(OBJECT_REUSE));
+        assertTrue(getConfiguration().get(OBJECT_REUSE));
     }
 
     @Test
@@ -109,22 +109,22 @@ public class SessionContextTest {
         // runtime config from flink-conf
         sessionContext.set(OBJECT_REUSE.key(), "false");
 
-        assertEquals("hive", getConfiguration().getString(TABLE_SQL_DIALECT));
-        assertEquals(128, getConfiguration().getInteger(MAX_PARALLELISM));
-        assertEquals("test", getConfiguration().getString(NAME));
-        assertFalse(getConfiguration().getBoolean(OBJECT_REUSE));
+        assertEquals("hive", getConfiguration().get(TABLE_SQL_DIALECT));
+        assertEquals(128, (int) getConfiguration().get(MAX_PARALLELISM));
+        assertEquals("test", getConfiguration().get(NAME));
+        assertFalse(getConfiguration().get(OBJECT_REUSE));
 
         sessionContext.reset(TABLE_SQL_DIALECT.key());
-        assertEquals("default", getConfiguration().getString(TABLE_SQL_DIALECT));
+        assertEquals("default", getConfiguration().get(TABLE_SQL_DIALECT));
 
         sessionContext.reset(MAX_PARALLELISM.key());
-        assertEquals(16, getConfiguration().getInteger(MAX_PARALLELISM));
+        assertEquals(16, (int) getConfiguration().get(MAX_PARALLELISM));
 
         sessionContext.reset(NAME.key());
         assertNull(getConfiguration().get(NAME));
 
         sessionContext.reset(OBJECT_REUSE.key());
-        assertTrue(getConfiguration().getBoolean(OBJECT_REUSE));
+        assertTrue(getConfiguration().get(OBJECT_REUSE));
     }
 
     @Test
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java
index 027f8da..6efa375 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/CompiledPlanImpl.java
@@ -54,10 +54,7 @@ class CompiledPlanImpl implements CompiledPlan {
         internalPlan.writeToFile(
                 file,
                 ignoreIfExists,
-                !tableEnvironment
-                        .getConfig()
-                        .getConfiguration()
-                        .get(TableConfigOptions.PLAN_FORCE_RECOMPILE));
+                !tableEnvironment.getConfig().get(TableConfigOptions.PLAN_FORCE_RECOMPILE));
     }
 
     @Override
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
index a692e55..91207f5 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
@@ -745,7 +745,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
                 return loadPlan(PlanReference.fromFile(filePath));
             }
 
-            if (!tableConfig.getConfiguration().get(TableConfigOptions.PLAN_FORCE_RECOMPILE)) {
+            if (!tableConfig.get(TableConfigOptions.PLAN_FORCE_RECOMPILE)) {
                 throw new TableException(
                         String.format(
                                 "Cannot overwrite the plan file '%s'. "
@@ -782,7 +782,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
         List<Transformation<?>> transformations = translate(operations);
         List<String> sinkIdentifierNames = extractSinkIdentifierNames(operations);
         TableResultInternal result = executeInternal(transformations, sinkIdentifierNames);
-        if (tableConfig.getConfiguration().get(TABLE_DML_SYNC)) {
+        if (tableConfig.get(TABLE_DML_SYNC)) {
             try {
                 result.await();
             } catch (InterruptedException | ExecutionException e) {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java
index b5fcf2d..e531da6 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/connectors/DynamicSourceUtils.java
@@ -237,9 +237,7 @@ public final class DynamicSourceUtils {
         boolean isCDCSource =
                 !mode.containsOnly(RowKind.INSERT) && !isUpsertSource(resolvedSchema, tableSource);
         boolean changeEventsDuplicate =
-                tableConfig
-                        .getConfiguration()
-                        .getBoolean(ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE);
+                tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE);
         boolean hasPrimaryKey = resolvedSchema.getPrimaryKey().isPresent();
         return isCDCSource && changeEventsDuplicate && hasPrimaryKey;
     }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java
index e3203cf..c158b0b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoSourceScanRuleBase.java
@@ -58,10 +58,8 @@ public abstract class PushFilterIntoSourceScanRuleBase extends RelOptRule {
     @Override
     public boolean matches(RelOptRuleCall call) {
         TableConfig tableConfig = ShortcutUtils.unwrapContext(call.getPlanner()).getTableConfig();
-        return tableConfig
-                .getConfiguration()
-                .getBoolean(
-                        OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED);
+        return tableConfig.get(
+                OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED);
     }
 
     protected List<RexNode> convertExpressionToRexNode(
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java
index 5d92e7c..06ee579 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushWatermarkIntoTableSourceScanRuleBase.java
@@ -109,9 +109,7 @@ public abstract class PushWatermarkIntoTableSourceScanRuleBase extends RelOptRul
             abilitySpec = sourceWatermarkSpec;
         } else {
             final Duration idleTimeout =
-                    tableConfig
-                            .getConfiguration()
-                            .get(ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT);
+                    tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SOURCE_IDLE_TIMEOUT);
             final long idleTimeoutMillis;
             if (!idleTimeout.isZero() && !idleTimeout.isNegative()) {
                 idleTimeoutMillis = idleTimeout.toMillis();
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java
index fe9a858..81bdaf6 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoScanRuleBase.java
@@ -78,10 +78,8 @@ public abstract class PushLocalAggIntoScanRuleBase extends RelOptRule {
             BatchPhysicalGroupAggregateBase aggregate,
             BatchPhysicalTableSourceScan tableSourceScan) {
         TableConfig tableConfig = ShortcutUtils.unwrapContext(call.getPlanner()).getTableConfig();
-        if (!tableConfig
-                .getConfiguration()
-                .getBoolean(
-                        OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED)) {
+        if (!tableConfig.get(
+                OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED)) {
             return false;
         }
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java
index d86937b..14fe596 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java
@@ -132,8 +132,7 @@ public final class CatalogSourceTable extends FlinkPreparingTableBase {
         if (hintedOptions.isEmpty()) {
             return contextResolvedTable;
         }
-        final ReadableConfig config = context.getTableConfig().getConfiguration();
-        if (!config.get(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) {
+        if (!context.getTableConfig().get(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) {
             throw new ValidationException(
                     String.format(
                             "The '%s' hint is allowed only when the config option '%s' is set to true.",
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
index 592dcf8..c7fa923 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
@@ -40,7 +40,7 @@ public class TableConfigUtils {
      * @return true if the given operator is disabled.
      */
     public static boolean isOperatorDisabled(TableConfig tableConfig, OperatorType operatorType) {
-        String value = tableConfig.getConfiguration().getString(TABLE_EXEC_DISABLED_OPERATORS);
+        String value = tableConfig.get(TABLE_EXEC_DISABLED_OPERATORS);
         if (value == null) {
             return false;
         }
@@ -68,8 +68,7 @@ public class TableConfigUtils {
      * @return the aggregate phase strategy
      */
     public static AggregatePhaseStrategy getAggPhaseStrategy(TableConfig tableConfig) {
-        String aggPhaseConf =
-                tableConfig.getConfiguration().getString(TABLE_OPTIMIZER_AGG_PHASE_STRATEGY).trim();
+        String aggPhaseConf = tableConfig.get(TABLE_OPTIMIZER_AGG_PHASE_STRATEGY).trim();
         if (aggPhaseConf.isEmpty()) {
             return AggregatePhaseStrategy.AUTO;
         } else {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala
index efc4bb0..10e37d3 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/BatchPlanner.scala
@@ -67,8 +67,7 @@ class BatchPlanner(
     // deadlock breakup
     processors.add(new DeadlockBreakupProcessor())
     // multiple input creation
-    if (getTableConfig.getConfiguration.getBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED)) {
+    if (getTableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED)) {
       processors.add(new MultipleInputNodeCreationProcessor(false))
     }
     processors.add(new ForwardHashExchangeProcessor)
@@ -154,7 +153,7 @@ class BatchPlanner(
 
   override def beforeTranslation(): Unit = {
     super.beforeTranslation()
-    val runtimeMode = getConfiguration.get(ExecutionOptions.RUNTIME_MODE)
+    val runtimeMode = getTableConfig.get(ExecutionOptions.RUNTIME_MODE)
     if (runtimeMode != RuntimeExecutionMode.BATCH) {
       throw new IllegalArgumentException(
         "Mismatch between configured runtime mode and actual runtime mode. " +
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
index c4e7aae..b31606e 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
@@ -494,7 +494,7 @@ abstract class PlannerBase(
       Thread.currentThread().getContextClassLoader)
 
     // Use config parallelism to override env parallelism.
-    val defaultParallelism = getTableConfig.getConfiguration.getInteger(
+    val defaultParallelism = getTableConfig.get(
       ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM)
     if (defaultParallelism > 0) {
       getExecEnv.getConfig.setParallelism(defaultParallelism)
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala
index 175e033..00fc1b9 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdColumnNullCount.scala
@@ -158,8 +158,7 @@ class FlinkRelMdColumnNullCount private extends MetadataHandler[ColumnNullCount]
         // If predicate has $index is not null, null count of index is must be 0 after predicate.
         val rexBuilder = rel.getCluster.getRexBuilder
         val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
-        val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
-          FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
+        val maxCnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
         val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate)
         val conjunctions = RelOptUtil.conjunctions(cnf)
         val notNullPredicatesAtIndexField = conjunctions.exists {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala
index a7a6c28..67b8a8c 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistinctRowCount.scala
@@ -204,8 +204,7 @@ class FlinkRelMdDistinctRowCount private extends MetadataHandler[BuiltInMetadata
     } else {
       val rexBuilder = rel.getCluster.getRexBuilder
       val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
-      val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
-        FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
+      val maxCnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
       val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate)
       val conjunctions = RelOptUtil.conjunctions(cnf)
       val conjunctionsWithoutExpandId = conjunctions.filterNot { c =>
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala
index 046e221..4e1dde7 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdDistribution.scala
@@ -72,8 +72,7 @@ class FlinkRelMdDistribution private extends MetadataHandler[FlinkDistribution]
 
   def flinkDistribution(sort: Sort, mq: RelMetadataQuery): FlinkRelDistribution = {
     val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(sort)
-    val enableRangeSort = tableConfig.getConfiguration.getBoolean(
-      BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED)
+    val enableRangeSort = tableConfig.get(BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED)
     if ((sort.getCollation.getFieldCollations.nonEmpty &&
       sort.fetch == null && sort.offset == null) && enableRangeSort) {
       //If Sort is global sort, and the table config allows the range partition.
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala
index 5b9ee74..5c39e66 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdRowCount.scala
@@ -176,8 +176,7 @@ class FlinkRelMdRowCount private extends MetadataHandler[BuiltInMetadata.RowCoun
       val inputRowCnt = mq.getRowCount(input)
       val config = rel.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
       val parallelism = (inputRowCnt /
-          config.getConfiguration.getLong(
-            FlinkRelMdRowCount.TABLE_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt
+          config.get(FlinkRelMdRowCount.TABLE_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt
       if (parallelism == 1) {
         ndvOfGroupKeysOnGlobalAgg
       } else if (grouping.isEmpty) {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala
index f01ab7e..34632e8 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/metadata/SelectivityEstimator.scala
@@ -57,8 +57,7 @@ class SelectivityEstimator(rel: RelNode, mq: FlinkRelMetadataQuery)
 
   private val rexBuilder = rel.getCluster.getRexBuilder
   private val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
-  private val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
-    FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
+  private val maxCnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
 
   // these default values is referred to RelMdUtil#guessSelectivity
   private[flink] val defaultComparisonSelectivity = Some(0.5d)
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala
index b0db94d..4b4f5b3 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/logical/FlinkLogicalSort.scala
@@ -105,8 +105,8 @@ class FlinkLogicalSortBatchConverter extends ConverterRule(
     val sort = rel.asInstanceOf[LogicalSort]
     val newInput = RelOptRule.convert(sort.getInput, FlinkConventions.LOGICAL)
     val config = sort.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
-    val enableRangeSort = config.getConfiguration.getBoolean(TABLE_EXEC_RANGE_SORT_ENABLED)
-    val limitValue = config.getConfiguration.getInteger(TABLE_EXEC_SORT_DEFAULT_LIMIT)
+    val enableRangeSort = config.get(TABLE_EXEC_RANGE_SORT_ENABLED)
+    val limitValue = config.get(TABLE_EXEC_SORT_DEFAULT_LIMIT)
     val (offset, fetch) = if (sort.fetch == null && sort.offset == null
       && !enableRangeSort && limitValue > 0) {
       //force the sort add limit
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala
index d5ebb60..ef2fae7 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala
@@ -110,7 +110,7 @@ class BatchPhysicalHashAggregate(
         } else {
           // If partialKey is enabled, try to use partial key to satisfy the required distribution
           val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-          val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
+          val partialKeyEnabled = tableConfig.get(
             BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
           partialKeyEnabled && groupKeysList.containsAll(shuffleKeys)
         }
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala
index 6c62e08..675e15e 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregateBase.scala
@@ -137,7 +137,7 @@ abstract class BatchPhysicalOverAggregateBase(
           val isAllFieldsFromInput = requiredDistribution.getKeys.forall(_ < inputFieldCnt)
           if (isAllFieldsFromInput) {
             val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-            if (tableConfig.getConfiguration.getBoolean(
+            if (tableConfig.get(
               BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)) {
               ImmutableIntList.of(partitionKeyIndices: _*).containsAll(requiredDistribution.getKeys)
             } else {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala
index 9d427c0..b5a2b07 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala
@@ -93,7 +93,7 @@ class BatchPhysicalPythonGroupAggregate(
         } else {
           // If partialKey is enabled, try to use partial key to satisfy the required distribution
           val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-          val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
+          val partialKeyEnabled = tableConfig.get(
             BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
           partialKeyEnabled && groupKeysList.containsAll(shuffleKeys)
         }
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala
index 4e8649f..900f155 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala
@@ -135,7 +135,7 @@ class BatchPhysicalRank(
         } else {
           // If partialKey is enabled, try to use partial key to satisfy the required distribution
           val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-          val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
+          val partialKeyEnabled = tableConfig.get(
             BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
           partialKeyEnabled && partitionKeyList.containsAll(shuffleKeys)
         }
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala
index 71b0fcf..cedc67b 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala
@@ -111,7 +111,7 @@ class BatchPhysicalSortAggregate(
         } else {
           // If partialKey is enabled, try to use partial key to satisfy the required distribution
           val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-          val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
+          val partialKeyEnabled = tableConfig.get(
             BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
           partialKeyEnabled && groupKeysList.containsAll(shuffleKeys)
         }
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala
index 7b351ee..df2c6e3 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkBatchProgram.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.planner.plan.optimize.program
 
-import org.apache.flink.configuration.Configuration
+import org.apache.flink.configuration.ReadableConfig
 import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
 import org.apache.flink.table.planner.plan.rules.FlinkBatchRuleSets
@@ -44,7 +44,7 @@ object FlinkBatchProgram {
   val PHYSICAL = "physical"
   val PHYSICAL_REWRITE = "physical_rewrite"
 
-  def buildProgram(config: Configuration): FlinkChainedProgram[BatchOptimizeContext] = {
+  def buildProgram(config: ReadableConfig): FlinkChainedProgram[BatchOptimizeContext] = {
     val chainedProgram = new FlinkChainedProgram[BatchOptimizeContext]()
 
     chainedProgram.addLast(
@@ -151,7 +151,7 @@ object FlinkBatchProgram {
         .build())
 
     // join reorder
-    if (config.getBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) {
+    if (config.get(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) {
       chainedProgram.addLast(
         JOIN_REORDER,
         FlinkGroupProgramBuilder.newBuilder[BatchOptimizeContext]
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala
index 422c67d..e34cc73 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala
@@ -832,7 +832,7 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti
       val inputChangelogMode = ChangelogPlanUtils.getChangelogMode(
         sink.getInput.asInstanceOf[StreamPhysicalRel]).get
       val primaryKeys = sink.contextResolvedTable.getResolvedSchema.getPrimaryKeyIndexes
-      val upsertMaterialize = tableConfig.getConfiguration.get(
+      val upsertMaterialize = tableConfig.get(
         ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE) match {
         case UpsertMaterialize.FORCE => primaryKeys.nonEmpty
         case UpsertMaterialize.NONE => false
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
index fa5f65c..c8a3fc5 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkStreamProgram.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.planner.plan.optimize.program
 
-import org.apache.flink.configuration.Configuration
+import org.apache.flink.configuration.ReadableConfig
 import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.planner.plan.nodes.FlinkConventions
 import org.apache.flink.table.planner.plan.rules.FlinkStreamRuleSets
@@ -43,7 +43,7 @@ object FlinkStreamProgram {
   val PHYSICAL = "physical"
   val PHYSICAL_REWRITE = "physical_rewrite"
 
-  def buildProgram(config: Configuration): FlinkChainedProgram[StreamOptimizeContext] = {
+  def buildProgram(config: ReadableConfig): FlinkChainedProgram[StreamOptimizeContext] = {
     val chainedProgram = new FlinkChainedProgram[StreamOptimizeContext]()
 
     // rewrite sub-queries to joins
@@ -161,7 +161,7 @@ object FlinkStreamProgram {
         .build())
 
     // join reorder
-    if (config.getBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) {
+    if (config.get(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED)) {
       chainedProgram.addLast(
         JOIN_REORDER,
         FlinkGroupProgramBuilder.newBuilder[StreamOptimizeContext]
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
index 4bb1f27..763f5b5 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/reuse/SubplanReuser.scala
@@ -56,11 +56,10 @@ object SubplanReuser {
     * Finds duplicated sub-plans and return the reused plan.
     */
   def reuseDuplicatedSubplan(rels: Seq[RelNode], tableConfig: TableConfig): Seq[RelNode] = {
-    if (!tableConfig.getConfiguration.getBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED)) {
+    if (!tableConfig.get(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED)) {
       return rels
     }
-    val tableSourceReuseEnabled = tableConfig.getConfiguration.getBoolean(
+    val tableSourceReuseEnabled = tableConfig.get(
       OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED)
     val context = new SubplanReuseContext(tableSourceReuseEnabled, rels: _*)
     val reuseShuttle = new SubplanReuseShuttle(context)
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala
index 099a147..069ffa8 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRule.scala
@@ -62,7 +62,7 @@ class JoinDeriveNullFilterRule
     val rexBuilder = join.getCluster.getRexBuilder
     val mq = FlinkRelMetadataQuery.reuseOrCreate(join.getCluster.getMetadataQuery)
     val conf = FlinkRelOptUtil.getTableConfigFromContext(join)
-    val minNullCount = conf.getConfiguration.getLong(
+    val minNullCount = conf.get(
       JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD)
 
     def createIsNotNullFilter(input: RelNode, keys: ImmutableIntList): RelNode = {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala
index b1b5270..25de4e5 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRule.scala
@@ -49,7 +49,7 @@ class PushFilterIntoLegacyTableSourceScanRule extends RelOptRule(
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val config = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
-    if (!config.getConfiguration.getBoolean(
+    if (!config.get(
       OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_PREDICATE_PUSHDOWN_ENABLED)) {
       return false
     }
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala
index 31d1f25..3963a4b 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRule.scala
@@ -128,7 +128,7 @@ class SplitAggregateRule extends RelOptRule(
     val tableConfig = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
     val agg: FlinkLogicalAggregate = call.rel(0)
 
-    val splitDistinctAggEnabled = tableConfig.getConfiguration.getBoolean(
+    val splitDistinctAggEnabled = tableConfig.get(
       OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED)
     val isAllAggSplittable = doAllAggSupportSplit(agg.getAggCallList)
 
@@ -165,7 +165,7 @@ class SplitAggregateRule extends RelOptRule(
     }.distinct.diff(aggGroupSet).sorted.toArray
 
     val hashFieldsMap: util.Map[Int, Int] = new util.HashMap()
-    val buckets = tableConfig.getConfiguration.getInteger(
+    val buckets = tableConfig.get(
       OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM)
 
     if (hashFieldIndexes.nonEmpty) {
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala
index f2b04ee..ea92321 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalHashJoinRule.scala
@@ -145,7 +145,7 @@ class BatchPhysicalHashJoinRule
         toHashTraitByColumns(joinInfo.rightKeys))
 
       // add more possibility to only shuffle by partial joinKeys, now only single one
-      val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean(
+      val isShuffleByPartialKeyEnabled = tableConfig.get(
         BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
       if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) {
         joinInfo.pairs().foreach { pair =>
@@ -177,7 +177,7 @@ class BatchPhysicalHashJoinRule
     if (leftSize == null || rightSize == null) {
       return (false, false)
     }
-    val threshold = tableConfig.getConfiguration.getLong(
+    val threshold = tableConfig.get(
       OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD)
     joinType match {
       case JoinRelType.LEFT => (rightSize <= threshold, false)
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala
index 14c0886..6e73cc9 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalJoinRuleBase.scala
@@ -56,7 +56,7 @@ trait BatchPhysicalJoinRuleBase {
       distinctKeys: Seq[Int]): Boolean = {
     val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(buildRel)
     val mq = buildRel.getCluster.getMetadataQuery
-    val ratioConf = tableConfig.getConfiguration.getDouble(
+    val ratioConf = tableConfig.get(
       BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SEMI_JOIN_BUILD_DISTINCT_NDV_RATIO)
     val inputRows = mq.getRowCount(buildRel)
     val ndvOfGroupKey = mq.getDistinctRowCount(
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala
index 9a72afe..321ca26 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortMergeJoinRule.scala
@@ -109,7 +109,7 @@ class BatchPhysicalSortMergeJoinRule
     }
 
     val tableConfig = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
-    val candidates = if (tableConfig.getConfiguration.getBoolean(
+    val candidates = if (tableConfig.get(
       BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED)) {
       // add more possibility to remove redundant sort, and longer optimization time
       Array((false, false), (true, false), (false, true), (true, true))
@@ -127,7 +127,7 @@ class BatchPhysicalSortMergeJoinRule
     }
 
     // add more possibility to only shuffle by partial joinKeys, now only single one
-    val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean(
+    val isShuffleByPartialKeyEnabled = tableConfig.get(
       BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED)
     if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) {
       joinInfo.pairs().foreach { pair =>
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala
index 6fc392b..bbd85cc 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/batch/BatchPhysicalSortRule.scala
@@ -53,7 +53,7 @@ class BatchPhysicalSortRule extends ConverterRule(
     val sort: FlinkLogicalSort = rel.asInstanceOf[FlinkLogicalSort]
     val input = sort.getInput
     val config = sort.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
-    val enableRangeSort = config.getConfiguration.getBoolean(
+    val enableRangeSort = config.get(
       BatchPhysicalSortRule.TABLE_EXEC_RANGE_SORT_ENABLED)
     val distribution = if (enableRangeSort) {
       FlinkRelDistribution.range(sort.getCollation.getFieldCollations)
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala
index 5aa549c..0e4ab86 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/IncrementalAggregateRule.scala
@@ -55,7 +55,7 @@ class IncrementalAggregateRule
     val tableConfig = call.getPlanner.getContext.unwrap(classOf[FlinkContext]).getTableConfig
 
     // whether incremental aggregate is enabled
-    val incrementalAggEnabled = tableConfig.getConfiguration.getBoolean(
+    val incrementalAggEnabled = tableConfig.get(
       IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED)
 
     partialGlobalAgg.partialFinalType == PartialFinalType.PARTIAL &&
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
index fff97ac..a8533e2 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
@@ -64,8 +64,7 @@ class MiniBatchIntervalInferRule extends RelOptRule(
     val miniBatchIntervalTrait = rel.getTraitSet.getTrait(MiniBatchIntervalTraitDef.INSTANCE)
     val inputs = getInputs(rel)
     val config = FlinkRelOptUtil.getTableConfigFromContext(rel)
-    val miniBatchEnabled = config.getConfiguration.getBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
+    val miniBatchEnabled = config.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
 
     val updatedTrait = rel match {
       case _: StreamPhysicalGroupWindowAggregate =>
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
index 1399bcf..e00b6ea 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
@@ -76,8 +76,7 @@ class TwoStageOptimizedAggregateRule extends RelOptRule(
       needRetraction,
       isStateBackendDataViews = true)
 
-    val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
+    val isMiniBatchEnabled = tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)
     val isTwoPhaseEnabled = getAggPhaseStrategy(tableConfig) != AggregatePhaseStrategy.ONE_PHASE
 
     isMiniBatchEnabled && isTwoPhaseEnabled &&
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala
index 0d38e57..3d44df5 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/schema/LegacyCatalogSourceTable.scala
@@ -81,11 +81,11 @@ class LegacyCatalogSourceTable[T](
       .unwrap(classOf[FlinkContext])
     val typeFactory = cluster.getTypeFactory.asInstanceOf[FlinkTypeFactory]
 
-    val conf = flinkContext.getTableConfig.getConfiguration
+    val conf = flinkContext.getTableConfig
 
     val hintedOptions = FlinkHints.getHintedOptions(context.getTableHints)
     if (hintedOptions.nonEmpty
-      && !conf.getBoolean(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) {
+      && !conf.get(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED)) {
       throw new ValidationException(s"${FlinkHints.HINT_NAME_OPTIONS} hint is allowed only when "
         + s"${TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED.key} "
         + s"is set to true")
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
index 7aa7524..bf8f67e 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/FlinkRelOptUtil.scala
@@ -200,8 +200,7 @@ object FlinkRelOptUtil {
 
   /** Get max cnf node limit by context of rel */
   def getMaxCnfNodeCount(rel: RelNode): Int = {
-    val tableConfig = getTableConfigFromContext(rel)
-    tableConfig.getConfiguration.getInteger(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
+    getTableConfigFromContext(rel).get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
   }
 
   /**
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala
index dbb60ff..303690b 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/IntervalJoinUtil.scala
@@ -74,8 +74,7 @@ object IntervalJoinUtil {
 
     // Converts the condition to conjunctive normal form (CNF)
     val cnfCondition = FlinkRexUtil.toCnf(rexBuilder,
-      tableConfig.getConfiguration.getInteger(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT),
-      predicate)
+      tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT), predicate)
 
     // split the condition into time predicates and other predicates
     // We need two range predicates or an equality predicate for a properly bounded window join.
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala
index ae9e522..77785ff 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/RankUtil.scala
@@ -75,8 +75,7 @@ object RankUtil {
       tableConfig: TableConfig): (Option[RankRange], Option[RexNode]) = {
     val predicate = FlinkRexUtil.expandSearch(rexBuilder, oriPred)
     // Converts the condition to conjunctive normal form (CNF)
-    val cnfNodeCount = tableConfig.getConfiguration.getInteger(
-      FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
+    val cnfNodeCount = tableConfig.get(FlinkRexUtil.TABLE_OPTIMIZER_CNF_NODES_LIMIT)
     val cnfCondition = FlinkRexUtil.toCnf(rexBuilder, cnfNodeCount, predicate)
 
     // split the condition into sort limit condition and other condition
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala
index 6a23c3b..69efd34 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/WindowUtil.scala
@@ -254,9 +254,8 @@ object WindowUtil {
    * throws exception when the configurations are set.
    */
   def checkEmitConfiguration(tableConfig: TableConfig): Unit = {
-    val conf = tableConfig.getConfiguration
-    if (conf.getBoolean(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED) ||
-      conf.getBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED)) {
+    if (tableConfig.get(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED) ||
+      tableConfig.get(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED)) {
       throw new TableException("Currently, window table function based aggregate doesn't " +
         s"support early-fire and late-fire configuration " +
         s"'${TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED.key()}' and " +
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala
index 88f1b6b..38b4a63 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/calcite/CalciteConfigBuilderTest.scala
@@ -38,7 +38,7 @@ class CalciteConfigBuilderTest {
     assertTrue(cc.getStreamProgram.isEmpty)
 
     val builder = new CalciteConfigBuilder()
-    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault.getConfiguration)
+    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault)
     streamPrograms.remove(FlinkStreamProgram.PHYSICAL)
     builder.replaceStreamProgram(streamPrograms)
 
@@ -174,7 +174,7 @@ class CalciteConfigBuilderTest {
     assertTrue(config.getSqlParserConfig.isEmpty)
     assertTrue(config.getSqlToRelConverterConfig.isEmpty)
 
-    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault.getConfiguration)
+    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault)
     streamPrograms.remove(FlinkStreamProgram.PHYSICAL)
     builder.replaceStreamProgram(streamPrograms)
     val baseConfig1 = builder.build()
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
index 6028eb4..70f0d9e 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
@@ -188,7 +188,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase {
          |FROM ($sql2) AS T
          |GROUP BY b
       """.stripMargin
-    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConfiguration)
+    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig)
     programs.remove(FlinkBatchProgram.PHYSICAL)
     testUtil.replaceBatchProgram(programs)
     testUtil.verifyRelPlan(sql)
@@ -196,7 +196,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase {
 
   @Test
   def testLogicalPlanWithImplicitTypeCast(): Unit = {
-    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConfiguration)
+    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig)
     programs.remove(FlinkBatchProgram.PHYSICAL)
     testUtil.replaceBatchProgram(programs)
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala
index 2df538e..05b8a5a 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/GroupWindowITCase.scala
@@ -583,9 +583,8 @@ class GroupWindowITCase(mode: StateBackendMode, useTimestampLtz: Boolean)
 
   private def withLateFireDelay(tableConfig: TableConfig, interval: Time): Unit = {
     val intervalInMillis = interval.toMilliseconds
-    val lateFireDelay: Duration = tableConfig.getConfiguration
-      .getOptional(TABLE_EXEC_EMIT_LATE_FIRE_DELAY)
-      .orElse(null)
+    val lateFireDelay: Duration =
+      tableConfig.getOptional(TABLE_EXEC_EMIT_LATE_FIRE_DELAY).orElse(null)
     if (lateFireDelay != null && (lateFireDelay.toMillis != intervalInMillis)) {
       // lateFireInterval of the two query config is not equal and not the default
       throw new RuntimeException(

[flink] 06/08: [hotfix][table-planner][tests] Use EnvironmentSettings to pass configuration

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 607c3a7ba415cc4138f446b43c48e06faae3e369
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 14:55:42 2022 +0200

    [hotfix][table-planner][tests] Use EnvironmentSettings to pass configuration
    
    Use the new way for passing application specific configuration through
    the `EnviromentSettings`.
    
    (cherry picked from commit d52658557c4ed5c7b7ffc44b1229da18b9c6aa0f)
---
 .../org/apache/flink/table/api/TableEnvironmentTest.scala     | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)

diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
index 3f9c906..9f2e803 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.api
 import org.apache.flink.api.common.RuntimeExecutionMode
 import org.apache.flink.api.common.typeinfo.Types.STRING
 import org.apache.flink.api.scala._
-import org.apache.flink.configuration.{Configuration, ExecutionOptions}
+import org.apache.flink.configuration.{Configuration, CoreOptions, ExecutionOptions}
 import org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches
 import org.apache.flink.streaming.api.environment.LocalStreamEnvironment
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
@@ -42,6 +42,7 @@ import org.apache.flink.types.Row
 import org.apache.calcite.plan.RelOptUtil
 import org.apache.calcite.sql.SqlExplainLevel
 import org.assertj.core.api.Assertions.{assertThat, assertThatThrownBy}
+
 import org.junit.Assert.{assertEquals, assertFalse, assertTrue, fail}
 import org.junit.rules.ExpectedException
 import org.junit.{Rule, Test}
@@ -142,11 +143,11 @@ class TableEnvironmentTest {
   @Test
   def testStreamTableEnvironmentExecutionExplainWithConfParallelism(): Unit = {
     val execEnv = StreamExecutionEnvironment.getExecutionEnvironment
-    val settings = EnvironmentSettings.newInstance().inStreamingMode().build()
-    val tEnv = StreamTableEnvironment.create(execEnv, settings)
     val configuration = new Configuration()
-    configuration.setInteger("parallelism.default", 4)
-    tEnv.getConfig.addConfiguration(configuration)
+    configuration.set(CoreOptions.DEFAULT_PARALLELISM, Integer.valueOf(4))
+    val settings =
+      EnvironmentSettings.newInstance().inStreamingMode().withConfiguration(configuration).build()
+    val tEnv = StreamTableEnvironment.create(execEnv, settings)
 
     verifyTableEnvironmentExecutionExplain(tEnv)
   }

[flink] 01/08: [hotfix][connector-hive] Avoid serializing TableConfig

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 0ba337cd9f1f6f374aba662769c0ff2b2013e49e
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 13:49:49 2022 +0200

    [hotfix][connector-hive] Avoid serializing TableConfig
    
    Use and pass around only `threadNum` which is the only option read,
    instead of the whole `TableConfig`, to prevent the relevant classes
    that are serialized from trying to serialize also the `TableConfig`.
    
    (cherry picked from commit 7fcd6935e065bb0b8dd0b7992ffb2887dab5a5f8)
---
 .../hive/ContinuousHiveSplitEnumerator.java        | 13 ++++-----
 .../connectors/hive/HiveLookupTableSource.java     |  2 +-
 .../apache/flink/connectors/hive/HiveSource.java   | 11 ++++----
 .../flink/connectors/hive/HiveSourceBuilder.java   | 18 ++++++++----
 .../connectors/hive/HiveSourceFileEnumerator.java  | 29 +++++++------------
 .../flink/connectors/hive/HiveTableSink.java       | 33 ++++++++++++++++++----
 .../flink/connectors/hive/HiveTableSource.java     |  4 ++-
 .../hive/read/HiveInputFormatPartitionReader.java  |  9 +++---
 .../connectors/hive/read/HiveTableInputFormat.java | 20 ++++---------
 .../connectors/hive/PartitionMonitorTest.java      |  3 +-
 .../read/HiveInputFormatPartitionReaderITCase.java |  4 +--
 11 files changed, 79 insertions(+), 67 deletions(-)

diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java
index 983faac..a061b27 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/ContinuousHiveSplitEnumerator.java
@@ -23,7 +23,6 @@ import org.apache.flink.api.connector.source.SourceEvent;
 import org.apache.flink.api.connector.source.SplitEnumerator;
 import org.apache.flink.api.connector.source.SplitEnumeratorContext;
 import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.connector.file.src.FileSourceSplit;
 import org.apache.flink.connector.file.src.PendingSplitsCheckpoint;
 import org.apache.flink.connector.file.src.assigners.FileSplitAssigner;
@@ -78,7 +77,7 @@ public class ContinuousHiveSplitEnumerator<T extends Comparable<T>>
             Collection<List<String>> seenPartitionsSinceOffset,
             FileSplitAssigner splitAssigner,
             long discoveryInterval,
-            ReadableConfig flinkConf,
+            int threadNum,
             JobConf jobConf,
             ObjectPath tablePath,
             ContinuousPartitionFetcher<Partition, T> fetcher,
@@ -95,7 +94,7 @@ public class ContinuousHiveSplitEnumerator<T extends Comparable<T>>
                         currentReadOffset,
                         seenPartitionsSinceOffset,
                         tablePath,
-                        flinkConf,
+                        threadNum,
                         jobConf,
                         fetcher,
                         fetcherContext);
@@ -188,7 +187,7 @@ public class ContinuousHiveSplitEnumerator<T extends Comparable<T>>
         private final Set<List<String>> seenPartitionsSinceOffset;
 
         private final ObjectPath tablePath;
-        private final ReadableConfig flinkConf;
+        private final int threadNum;
         private final JobConf jobConf;
         private final ContinuousPartitionFetcher<Partition, T> fetcher;
         private final HiveContinuousPartitionContext<Partition, T> fetcherContext;
@@ -197,14 +196,14 @@ public class ContinuousHiveSplitEnumerator<T extends Comparable<T>>
                 T currentReadOffset,
                 Collection<List<String>> seenPartitionsSinceOffset,
                 ObjectPath tablePath,
-                ReadableConfig flinkConf,
+                int threadNum,
                 JobConf jobConf,
                 ContinuousPartitionFetcher<Partition, T> fetcher,
                 HiveContinuousPartitionContext<Partition, T> fetcherContext) {
             this.currentReadOffset = currentReadOffset;
             this.seenPartitionsSinceOffset = new HashSet<>(seenPartitionsSinceOffset);
             this.tablePath = tablePath;
-            this.flinkConf = flinkConf;
+            this.threadNum = threadNum;
             this.jobConf = jobConf;
             this.fetcher = fetcher;
             this.fetcherContext = fetcherContext;
@@ -244,7 +243,7 @@ public class ContinuousHiveSplitEnumerator<T extends Comparable<T>>
                                     0,
                                     Collections.singletonList(
                                             fetcherContext.toHiveTablePartition(partition)),
-                                    flinkConf,
+                                    threadNum,
                                     jobConf));
                 }
             }
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java
index 488ea1b..e68072a 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveLookupTableSource.java
@@ -242,7 +242,7 @@ public class HiveLookupTableSource extends HiveTableSource implements LookupTabl
 
         PartitionReader<HiveTablePartition, RowData> partitionReader =
                 new HiveInputFormatPartitionReader(
-                        flinkConf,
+                        flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM),
                         jobConf,
                         hiveVersion,
                         tablePath,
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java
index 26c4ac5..d7f5209 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSource.java
@@ -22,7 +22,6 @@ import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.connector.source.Boundedness;
 import org.apache.flink.api.connector.source.SplitEnumerator;
 import org.apache.flink.api.connector.source.SplitEnumeratorContext;
-import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.connector.file.src.AbstractFileSource;
 import org.apache.flink.connector.file.src.ContinuousEnumerationSettings;
 import org.apache.flink.connector.file.src.PendingSplitsCheckpoint;
@@ -60,7 +59,7 @@ public class HiveSource<T> extends AbstractFileSource<T, HiveSourceSplit> {
 
     private static final long serialVersionUID = 1L;
 
-    private final ReadableConfig flinkConf;
+    private final int threadNum;
     private final JobConfWrapper jobConfWrapper;
     private final List<String> partitionKeys;
     private final ContinuousPartitionFetcher<Partition, ?> fetcher;
@@ -73,7 +72,7 @@ public class HiveSource<T> extends AbstractFileSource<T, HiveSourceSplit> {
             FileSplitAssigner.Provider splitAssigner,
             BulkFormat<T, HiveSourceSplit> readerFormat,
             @Nullable ContinuousEnumerationSettings continuousEnumerationSettings,
-            ReadableConfig flinkConf,
+            int threadNum,
             JobConf jobConf,
             ObjectPath tablePath,
             List<String> partitionKeys,
@@ -86,10 +85,10 @@ public class HiveSource<T> extends AbstractFileSource<T, HiveSourceSplit> {
                 readerFormat,
                 continuousEnumerationSettings);
         Preconditions.checkArgument(
-                flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM) >= 1,
+                threadNum >= 1,
                 HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM.key()
                         + " cannot be less than 1");
-        this.flinkConf = flinkConf;
+        this.threadNum = threadNum;
         this.jobConfWrapper = new JobConfWrapper(jobConf);
         this.tablePath = tablePath;
         this.partitionKeys = partitionKeys;
@@ -164,7 +163,7 @@ public class HiveSource<T> extends AbstractFileSource<T, HiveSourceSplit> {
                 seenPartitions,
                 getAssignerFactory().create(new ArrayList<>(splits)),
                 getContinuousEnumerationSettings().getDiscoveryInterval().toMillis(),
-                flinkConf,
+                threadNum,
                 jobConfWrapper.conf(),
                 tablePath,
                 fetcher,
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java
index 25f7ed7..75b6e74 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceBuilder.java
@@ -68,6 +68,7 @@ import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_ENAB
 import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_MONITOR_INTERVAL;
 import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_PARTITION_INCLUDE;
 import static org.apache.flink.connectors.hive.HiveOptions.STREAMING_SOURCE_PARTITION_ORDER;
+import static org.apache.flink.connectors.hive.HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER;
 import static org.apache.flink.table.catalog.hive.util.HiveTableUtil.checkAcidTable;
 import static org.apache.flink.util.Preconditions.checkArgument;
 
@@ -78,7 +79,8 @@ public class HiveSourceBuilder {
     private static final Duration DEFAULT_SCAN_MONITOR_INTERVAL = Duration.ofMinutes(1L);
 
     private final JobConf jobConf;
-    private final ReadableConfig flinkConf;
+    private final int threadNum;
+    private final boolean fallbackMappedReader;
 
     private final ObjectPath tablePath;
     private final Map<String, String> tableOptions;
@@ -110,7 +112,9 @@ public class HiveSourceBuilder {
             @Nonnull String tableName,
             @Nonnull Map<String, String> tableOptions) {
         this.jobConf = jobConf;
-        this.flinkConf = flinkConf;
+        this.threadNum =
+                flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM);
+        this.fallbackMappedReader = flinkConf.get(TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER);
         this.tablePath = new ObjectPath(dbName, tableName);
         this.hiveVersion = hiveVersion == null ? HiveShimLoader.getHiveVersion() : hiveVersion;
         HiveConf hiveConf = HiveConfUtils.create(jobConf);
@@ -147,7 +151,9 @@ public class HiveSourceBuilder {
             @Nullable String hiveVersion,
             @Nonnull CatalogTable catalogTable) {
         this.jobConf = jobConf;
-        this.flinkConf = flinkConf;
+        this.threadNum =
+                flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM);
+        this.fallbackMappedReader = flinkConf.get(TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER);
         this.tablePath = tablePath;
         this.hiveVersion = hiveVersion == null ? HiveShimLoader.getHiveVersion() : hiveVersion;
         this.fullSchema = catalogTable.getSchema();
@@ -231,12 +237,12 @@ public class HiveSourceBuilder {
                 new Path[1],
                 new HiveSourceFileEnumerator.Provider(
                         partitions != null ? partitions : Collections.emptyList(),
-                        flinkConf,
+                        threadNum,
                         new JobConfWrapper(jobConf)),
                 splitAssigner,
                 bulkFormat,
                 continuousSourceSettings,
-                flinkConf,
+                threadNum,
                 jobConf,
                 tablePath,
                 partitionKeys,
@@ -318,7 +324,7 @@ public class HiveSourceBuilder {
                         fullSchema.getFieldDataTypes(),
                         hiveVersion,
                         getProducedRowType(),
-                        flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER)),
+                        fallbackMappedReader),
                 limit);
     }
 }
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java
index 4227c36..dd0fe98 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveSourceFileEnumerator.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.connectors.hive;
 
-import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.connector.file.src.FileSourceSplit;
 import org.apache.flink.connector.file.src.enumerate.FileEnumerator;
 import org.apache.flink.connectors.hive.read.HiveSourceSplit;
@@ -45,33 +44,27 @@ public class HiveSourceFileEnumerator implements FileEnumerator {
     // For non-partition hive table, partitions only contains one partition which partitionValues is
     // empty.
     private final List<HiveTablePartition> partitions;
-    private final ReadableConfig flinkConf;
+    private final int threadNum;
     private final JobConf jobConf;
 
     public HiveSourceFileEnumerator(
-            List<HiveTablePartition> partitions, ReadableConfig flinkConf, JobConf jobConf) {
+            List<HiveTablePartition> partitions, int threadNum, JobConf jobConf) {
         this.partitions = partitions;
-        this.flinkConf = flinkConf;
+        this.threadNum = threadNum;
         this.jobConf = jobConf;
     }
 
     @Override
     public Collection<FileSourceSplit> enumerateSplits(Path[] paths, int minDesiredSplits)
             throws IOException {
-        return new ArrayList<>(createInputSplits(minDesiredSplits, partitions, flinkConf, jobConf));
+        return new ArrayList<>(createInputSplits(minDesiredSplits, partitions, threadNum, jobConf));
     }
 
     public static List<HiveSourceSplit> createInputSplits(
-            int minNumSplits,
-            List<HiveTablePartition> partitions,
-            ReadableConfig flinkConf,
-            JobConf jobConf)
+            int minNumSplits, List<HiveTablePartition> partitions, int threadNum, JobConf jobConf)
             throws IOException {
         List<HiveSourceSplit> hiveSplits = new ArrayList<>();
-        try (MRSplitsGetter splitsGetter =
-                new MRSplitsGetter(
-                        flinkConf.get(
-                                HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM))) {
+        try (MRSplitsGetter splitsGetter = new MRSplitsGetter(threadNum)) {
             for (HiveTablePartitionSplits partitionSplits :
                     splitsGetter.getHiveTablePartitionMRSplits(minNumSplits, partitions, jobConf)) {
                 HiveTablePartition partition = partitionSplits.getHiveTablePartition();
@@ -109,21 +102,19 @@ public class HiveSourceFileEnumerator implements FileEnumerator {
         private static final long serialVersionUID = 1L;
 
         private final List<HiveTablePartition> partitions;
-        private final ReadableConfig flinkConf;
+        private final int threadNum;
         private final JobConfWrapper jobConfWrapper;
 
         public Provider(
-                List<HiveTablePartition> partitions,
-                ReadableConfig flinkConf,
-                JobConfWrapper jobConfWrapper) {
+                List<HiveTablePartition> partitions, int threadNum, JobConfWrapper jobConfWrapper) {
             this.partitions = partitions;
-            this.flinkConf = flinkConf;
+            this.threadNum = threadNum;
             this.jobConfWrapper = jobConfWrapper;
         }
 
         @Override
         public FileEnumerator create() {
-            return new HiveSourceFileEnumerator(partitions, flinkConf, jobConfWrapper.conf());
+            return new HiveSourceFileEnumerator(partitions, threadNum, jobConfWrapper.conf());
         }
     }
 }
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java
index 710c890..4316670 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSink.java
@@ -108,7 +108,8 @@ public class HiveTableSink implements DynamicTableSink, SupportsPartitioning, Su
 
     private static final Logger LOG = LoggerFactory.getLogger(HiveTableSink.class);
 
-    private final ReadableConfig flinkConf;
+    private final boolean fallbackMappedReader;
+    private final boolean fallbackMappedWriter;
     private final JobConf jobConf;
     private final CatalogTable catalogTable;
     private final ObjectIdentifier identifier;
@@ -128,7 +129,24 @@ public class HiveTableSink implements DynamicTableSink, SupportsPartitioning, Su
             ObjectIdentifier identifier,
             CatalogTable table,
             @Nullable Integer configuredParallelism) {
-        this.flinkConf = flinkConf;
+        this(
+                flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER),
+                flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER),
+                jobConf,
+                identifier,
+                table,
+                configuredParallelism);
+    }
+
+    private HiveTableSink(
+            boolean fallbackMappedReader,
+            boolean fallbackMappedWriter,
+            JobConf jobConf,
+            ObjectIdentifier identifier,
+            CatalogTable table,
+            @Nullable Integer configuredParallelism) {
+        this.fallbackMappedReader = fallbackMappedReader;
+        this.fallbackMappedWriter = fallbackMappedWriter;
         this.jobConf = jobConf;
         this.identifier = identifier;
         this.catalogTable = table;
@@ -298,7 +316,7 @@ public class HiveTableSink implements DynamicTableSink, SupportsPartitioning, Su
         org.apache.flink.core.fs.Path path = new org.apache.flink.core.fs.Path(sd.getLocation());
 
         BucketsBuilder<RowData, String, ? extends BucketsBuilder<RowData, ?, ?>> builder;
-        if (flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER)) {
+        if (fallbackMappedWriter) {
             builder =
                     bucketsBuilderForMRWriter(
                             recordWriterFactory, sd, assigner, rollingPolicy, outputFileConfig);
@@ -377,7 +395,7 @@ public class HiveTableSink implements DynamicTableSink, SupportsPartitioning, Su
                 catalogTable,
                 hiveVersion,
                 (RowType) tableSchema.toRowDataType().getLogicalType(),
-                flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER));
+                fallbackMappedReader);
     }
 
     private HiveTableMetaStoreFactory msFactory() {
@@ -487,7 +505,12 @@ public class HiveTableSink implements DynamicTableSink, SupportsPartitioning, Su
     public DynamicTableSink copy() {
         HiveTableSink sink =
                 new HiveTableSink(
-                        flinkConf, jobConf, identifier, catalogTable, configuredParallelism);
+                        fallbackMappedReader,
+                        fallbackMappedWriter,
+                        jobConf,
+                        identifier,
+                        catalogTable,
+                        configuredParallelism);
         sink.staticPartitionSpec = staticPartitionSpec;
         sink.overwrite = overwrite;
         sink.dynamicGrouping = dynamicGrouping;
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
index d7ee578..cc5d639 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/HiveTableSource.java
@@ -146,6 +146,8 @@ public class HiveTableSource
                             catalogTable.getPartitionKeys(),
                             remainingPartitions);
 
+            int threadNum =
+                    flinkConf.get(HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM);
             int parallelism =
                     new HiveParallelismInference(tablePath, flinkConf)
                             .infer(
@@ -156,7 +158,7 @@ public class HiveTableSource
                                             HiveSourceFileEnumerator.createInputSplits(
                                                             0,
                                                             hivePartitionsToRead,
-                                                            flinkConf,
+                                                            threadNum,
                                                             jobConf)
                                                     .size())
                             .limit(limit);
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java
index 534c623..2493942 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReader.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.connectors.hive.read;
 
-import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.connector.file.table.PartitionReader;
 import org.apache.flink.connectors.hive.HiveTablePartition;
 import org.apache.flink.connectors.hive.JobConfWrapper;
@@ -36,7 +35,7 @@ public class HiveInputFormatPartitionReader
         implements PartitionReader<HiveTablePartition, RowData> {
 
     private static final long serialVersionUID = 1L;
-    private final ReadableConfig flinkConf;
+    private final int threadNum;
     private final JobConfWrapper jobConfWrapper;
     private final String hiveVersion;
     protected final ObjectPath tablePath;
@@ -51,7 +50,7 @@ public class HiveInputFormatPartitionReader
     private transient int readingSplitId;
 
     public HiveInputFormatPartitionReader(
-            ReadableConfig flinkConf,
+            int threadNum,
             JobConf jobConf,
             String hiveVersion,
             ObjectPath tablePath,
@@ -60,7 +59,7 @@ public class HiveInputFormatPartitionReader
             List<String> partitionKeys,
             int[] selectedFields,
             boolean useMapRedReader) {
-        this.flinkConf = flinkConf;
+        this.threadNum = threadNum;
         this.jobConfWrapper = new JobConfWrapper(jobConf);
         this.hiveVersion = hiveVersion;
         this.tablePath = tablePath;
@@ -75,7 +74,7 @@ public class HiveInputFormatPartitionReader
     public void open(List<HiveTablePartition> partitions) throws IOException {
         hiveTableInputFormat =
                 new HiveTableInputFormat(
-                        this.flinkConf,
+                        this.threadNum,
                         this.jobConfWrapper.conf(),
                         this.partitionKeys,
                         this.fieldTypes,
diff --git a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java
index 5297812..eadd114 100644
--- a/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java
+++ b/flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/connectors/hive/read/HiveTableInputFormat.java
@@ -23,8 +23,6 @@ import org.apache.flink.api.common.io.CheckpointableInputFormat;
 import org.apache.flink.api.common.io.LocatableInputSplitAssigner;
 import org.apache.flink.api.common.io.statistics.BaseStatistics;
 import org.apache.flink.api.java.hadoop.common.HadoopInputFormatCommonBase;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.connectors.hive.HiveOptions;
 import org.apache.flink.connectors.hive.HiveTablePartition;
 import org.apache.flink.connectors.hive.HiveTablePartitionSplits;
 import org.apache.flink.connectors.hive.JobConfWrapper;
@@ -69,7 +67,7 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase<RowData, H
     private static final String SCHEMA_EVOLUTION_COLUMNS = "schema.evolution.columns";
     private static final String SCHEMA_EVOLUTION_COLUMNS_TYPES = "schema.evolution.columns.types";
 
-    private final ReadableConfig flinkConf;
+    private final int threadNum;
 
     private final JobConfWrapper jobConf;
 
@@ -98,7 +96,7 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase<RowData, H
     @VisibleForTesting protected transient SplitReader reader;
 
     public HiveTableInputFormat(
-            ReadableConfig flinkConf,
+            int threadNum,
             JobConf jobConf,
             List<String> partitionKeys,
             DataType[] fieldTypes,
@@ -109,7 +107,7 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase<RowData, H
             boolean useMapRedReader,
             List<HiveTablePartition> partitions) {
         super(jobConf.getCredentials());
-        this.flinkConf = flinkConf;
+        this.threadNum = threadNum;
         this.jobConf = new JobConfWrapper(new JobConf(jobConf));
         this.partitionKeys = partitionKeys;
         this.fieldTypes = fieldTypes;
@@ -317,21 +315,15 @@ public class HiveTableInputFormat extends HadoopInputFormatCommonBase<RowData, H
 
     @Override
     public HiveTableInputSplit[] createInputSplits(int minNumSplits) throws IOException {
-        return createInputSplits(minNumSplits, partitions, flinkConf, jobConf.conf());
+        return createInputSplits(minNumSplits, partitions, threadNum, jobConf.conf());
     }
 
     public static HiveTableInputSplit[] createInputSplits(
-            int minNumSplits,
-            List<HiveTablePartition> partitions,
-            ReadableConfig flinkConf,
-            JobConf jobConf)
+            int minNumSplits, List<HiveTablePartition> partitions, int threadNum, JobConf jobConf)
             throws IOException {
         List<HiveTableInputSplit> hiveSplits = new ArrayList<>();
         int splitNum = 0;
-        try (MRSplitsGetter splitsGetter =
-                new MRSplitsGetter(
-                        flinkConf.get(
-                                HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM))) {
+        try (MRSplitsGetter splitsGetter = new MRSplitsGetter(threadNum)) {
             for (HiveTablePartitionSplits partitionSplits :
                     splitsGetter.getHiveTablePartitionMRSplits(minNumSplits, partitions, jobConf)) {
                 for (InputSplit inputSplit : partitionSplits.getInputSplits()) {
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java
index f366f10..0816092 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/PartitionMonitorTest.java
@@ -182,7 +182,8 @@ public class PartitionMonitorTest {
                         0L,
                         seenPartitionsSinceOffset,
                         tablePath,
-                        configuration,
+                        configuration.get(
+                                HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM),
                         jobConf,
                         continuousPartitionFetcher,
                         fetcherContext);
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java
index bc8b971..aac9007 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/read/HiveInputFormatPartitionReaderITCase.java
@@ -17,7 +17,7 @@
 
 package org.apache.flink.connectors.hive.read;
 
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connectors.hive.HiveOptions;
 import org.apache.flink.connectors.hive.HiveTablePartition;
 import org.apache.flink.table.api.SqlDialect;
 import org.apache.flink.table.api.TableEnvironment;
@@ -62,7 +62,7 @@ public class HiveInputFormatPartitionReaderITCase {
         // create partition reader
         HiveInputFormatPartitionReader partitionReader =
                 new HiveInputFormatPartitionReader(
-                        new Configuration(),
+                        HiveOptions.TABLE_EXEC_HIVE_LOAD_PARTITION_SPLITS_THREAD_NUM.defaultValue(),
                         new JobConf(hiveCatalog.getHiveConf()),
                         hiveCatalog.getHiveVersion(),
                         tablePath,

[flink] 07/08: [hotfix][table-planner] Remove ExecNodeConfig#getLocalTimeZone

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 30ce8e72baa8dad98d4901eebe93092d002aea06
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 16:56:01 2022 +0200

    [hotfix][table-planner] Remove ExecNodeConfig#getLocalTimeZone
    
    Remove `ExecNodeConfig#getLocalTimeZone` by replacing its usages with
    the newly introduced method in the utility class `TableConfigUtils`.
    
    (cherry picked from commit bc0cc0c724dd3ebddbcd1d5a5c5465db1ce8c621)
---
 .../planner/plan/nodes/exec/ExecNodeConfig.java    |  7 -----
 .../exec/common/CommonExecWindowTableFunction.java |  4 ++-
 .../stream/StreamExecGlobalWindowAggregate.java    |  4 ++-
 .../stream/StreamExecGroupWindowAggregate.java     |  3 +-
 .../stream/StreamExecLocalWindowAggregate.java     |  4 ++-
 .../StreamExecPythonGroupWindowAggregate.java      |  3 +-
 .../exec/stream/StreamExecWindowAggregate.java     |  4 ++-
 .../exec/stream/StreamExecWindowDeduplicate.java   |  4 ++-
 .../nodes/exec/stream/StreamExecWindowJoin.java    |  4 ++-
 .../nodes/exec/stream/StreamExecWindowRank.java    |  4 ++-
 .../table/planner/utils/TableConfigUtils.java      | 33 ++++++++++++++++++++++
 11 files changed, 58 insertions(+), 16 deletions(-)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
index b19ec9f..13df31b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
@@ -27,7 +27,6 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.planner.delegation.PlannerBase;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
 
-import java.time.ZoneId;
 import java.util.Optional;
 
 /**
@@ -94,10 +93,4 @@ public final class ExecNodeConfig implements ReadableConfig {
     public long getMaxIdleStateRetentionTime() {
         return tableConfig.getMaxIdleStateRetentionTime();
     }
-
-    // See https://issues.apache.org/jira/browse/FLINK-26190
-    /** See {@link TableConfig#getLocalTimeZone()}. */
-    public ZoneId getLocalTimeZone() {
-        return tableConfig.getLocalTimeZone();
-    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
index fd46ec3..998ab8e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
@@ -31,6 +31,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNode;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.operators.window.TimeWindow;
 import org.apache.flink.table.runtime.operators.window.WindowTableFunctionOperator;
 import org.apache.flink.table.runtime.operators.window.assigners.WindowAssigner;
@@ -80,7 +81,8 @@ public abstract class CommonExecWindowTableFunction extends ExecNodeBase<RowData
         WindowAssigner<TimeWindow> windowAssigner = createWindowAssigner(windowingStrategy);
         final ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
-                        windowingStrategy.getTimeAttributeType(), config.getLocalTimeZone());
+                        windowingStrategy.getTimeAttributeType(),
+                        TableConfigUtils.getLocalTimeZone(config));
         WindowTableFunctionOperator windowTableFunctionOperator =
                 new WindowTableFunctionOperator(
                         windowAssigner, windowingStrategy.getTimeAttributeIndex(), shiftTimeZone);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
index d3c2710..e4a3ec7 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
@@ -39,6 +39,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
 import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty;
 import org.apache.flink.table.runtime.groupwindow.WindowProperty;
@@ -151,7 +152,8 @@ public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBa
 
         final ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
-                        windowing.getTimeAttributeType(), config.getLocalTimeZone());
+                        windowing.getTimeAttributeType(),
+                        TableConfigUtils.getLocalTimeZone(config));
         final SliceAssigner sliceAssigner = createSliceAssigner(windowing, shiftTimeZone);
 
         final AggregateInfoList localAggInfoList =
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
index 0e33097..3944e4b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
@@ -46,6 +46,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedClass;
 import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
 import org.apache.flink.table.runtime.generated.GeneratedNamespaceTableAggsHandleFunction;
@@ -226,7 +227,7 @@ public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase {
         final ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
                         window.timeAttribute().getOutputDataType().getLogicalType(),
-                        config.getLocalTimeZone());
+                        TableConfigUtils.getLocalTimeZone(config));
 
         final boolean[] aggCallNeedRetractions = new boolean[aggCalls.length];
         Arrays.fill(aggCallNeedRetractions, needRetraction);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
index a701962..0c17ac7 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
@@ -38,6 +38,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.operators.aggregate.window.LocalSlicingWindowAggOperator;
@@ -137,7 +138,8 @@ public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBas
 
         final ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
-                        windowing.getTimeAttributeType(), config.getLocalTimeZone());
+                        windowing.getTimeAttributeType(),
+                        TableConfigUtils.getLocalTimeZone(config));
         final SliceAssigner sliceAssigner = createSliceAssigner(windowing, shiftTimeZone);
 
         final AggregateInfoList aggInfoList =
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
index ea153dd..2ebc5e2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
@@ -53,6 +53,7 @@ import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.PythonUtil;
 import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.dataview.DataViewSpec;
 import org.apache.flink.table.runtime.generated.GeneratedProjection;
 import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty;
@@ -224,7 +225,7 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas
         final ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
                         window.timeAttribute().getOutputDataType().getLogicalType(),
-                        config.getLocalTimeZone());
+                        TableConfigUtils.getLocalTimeZone(config));
         Tuple2<WindowAssigner<?>, Trigger<?>> windowAssignerAndTrigger =
                 generateWindowAssignerAndTrigger();
         WindowAssigner<?> windowAssigner = windowAssignerAndTrigger.f0;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
index 094c70c..b5a50bf 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
@@ -39,6 +39,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedNamespaceAggsHandleFunction;
 import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty;
 import org.apache.flink.table.runtime.groupwindow.WindowProperty;
@@ -152,7 +153,8 @@ public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase {
 
         final ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
-                        windowing.getTimeAttributeType(), config.getLocalTimeZone());
+                        windowing.getTimeAttributeType(),
+                        TableConfigUtils.getLocalTimeZone(config));
         final SliceAssigner sliceAssigner = createSliceAssigner(windowing, shiftTimeZone);
 
         // Hopping window requires additional COUNT(*) to determine whether to register next timer
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
index b0ddee0..35091f7 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
@@ -38,6 +38,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
 import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.operators.deduplicate.window.RowTimeWindowDeduplicateOperatorBuilder;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -149,7 +150,8 @@ public class StreamExecWindowDeduplicate extends ExecNodeBase<RowData>
 
         ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
-                        windowing.getTimeAttributeType(), config.getLocalTimeZone());
+                        windowing.getTimeAttributeType(),
+                        TableConfigUtils.getLocalTimeZone(config));
 
         RowType inputType = (RowType) inputEdge.getOutputType();
         RowDataKeySelector selector =
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
index e87cb3f..1b726d2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
@@ -38,6 +38,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.JoinUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.operators.join.window.WindowJoinOperator;
@@ -161,7 +162,8 @@ public class StreamExecWindowJoin extends ExecNodeBase<RowData>
 
         ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
-                        leftWindowing.getTimeAttributeType(), config.getLocalTimeZone());
+                        leftWindowing.getTimeAttributeType(),
+                        TableConfigUtils.getLocalTimeZone(config));
         WindowJoinOperator operator =
                 WindowJoinOperatorBuilder.builder()
                         .leftSerializer(leftTypeInfo.toRowSerializer())
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
index 73f752c..2cfb5d8 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
@@ -41,6 +41,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.operators.rank.ConstantRankRange;
@@ -218,7 +219,8 @@ public class StreamExecWindowRank extends ExecNodeBase<RowData>
 
         ZoneId shiftTimeZone =
                 TimeWindowUtil.getShiftTimeZone(
-                        windowing.getTimeAttributeType(), config.getLocalTimeZone());
+                        windowing.getTimeAttributeType(),
+                        TableConfigUtils.getLocalTimeZone(config));
         GeneratedRecordComparator sortKeyComparator =
                 ComparatorCodeGenerator.gen(
                         config.getTableConfig(),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
index c7fa923..02f00ca 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
@@ -18,14 +18,18 @@
 
 package org.apache.flink.table.planner.utils;
 
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.planner.calcite.CalciteConfig;
 import org.apache.flink.table.planner.calcite.CalciteConfig$;
 import org.apache.flink.table.planner.plan.utils.OperatorType;
 
+import java.time.ZoneId;
 import java.util.HashSet;
 import java.util.Set;
 
+import static java.time.ZoneId.SHORT_IDS;
 import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS;
 import static org.apache.flink.table.api.config.OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY;
 
@@ -89,6 +93,35 @@ public class TableConfigUtils {
                 .orElse(CalciteConfig$.MODULE$.DEFAULT());
     }
 
+    /**
+     * Similar to {@link TableConfig#getLocalTimeZone()} but extracting it from a generic {@link
+     * ReadableConfig}.
+     *
+     * @see TableConfig#getLocalTimeZone()
+     */
+    public static ZoneId getLocalTimeZone(ReadableConfig tableConfig) {
+        String zone = tableConfig.get(TableConfigOptions.LOCAL_TIME_ZONE);
+        validateTimeZone(zone);
+        return TableConfigOptions.LOCAL_TIME_ZONE.defaultValue().equals(zone)
+                ? ZoneId.systemDefault()
+                : ZoneId.of(zone);
+    }
+
+    /** Validates user configured time zone. */
+    private static void validateTimeZone(String zone) {
+        final String zoneId = zone.toUpperCase();
+        if (zoneId.startsWith("UTC+")
+                || zoneId.startsWith("UTC-")
+                || SHORT_IDS.containsKey(zoneId)) {
+            throw new IllegalArgumentException(
+                    String.format(
+                            "The supported Zone ID is either a full name such as "
+                                    + "'America/Los_Angeles', or a custom timezone id such as "
+                                    + "'GMT-08:00', but configured Zone ID is '%s'.",
+                            zone));
+        }
+    }
+
     // Make sure that we cannot instantiate this class
     private TableConfigUtils() {}
 }

[flink] 04/08: [FLINK-26709][table] Replace TableConfig.getConfiguration

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 885d4bb3c2016cb97e9d6191c1047ca60ed8cfde
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 14:49:16 2022 +0200

    [FLINK-26709][table] Replace TableConfig.getConfiguration
    
    Replace `TableConfig.getConfiguration()` with directly passing
    `TableConfig`, since `TableConfig` is now a `ReadableConfig` and
     the calls to `get/getOptional` give a full view, including the
     `rootConfiguration`, which makes all the options coming from the
    environment (flink-conf.yaml, CLI params) available.
    
    (cherry picked from commit 09f2c59ee35c6d4e714bfa90b7f04a1c8ddf4bae)
---
 .../connectors/hive/HiveDynamicTableFactoryTest.java     |  4 ++--
 .../flink/connectors/hive/HiveLookupJoinITCase.java      |  2 +-
 .../flink/client/python/PythonFunctionFactory.java       |  7 ++++++-
 .../apache/flink/table/client/cli/CliResultViewTest.java |  4 +---
 .../apache/flink/table/client/cli/TestingExecutor.java   |  3 +--
 .../table/client/gateway/context/SessionContextTest.java |  9 +++------
 .../internal/AbstractStreamTableEnvironmentImpl.java     |  3 ++-
 .../java/org/apache/flink/table/api/TableResult.java     |  4 ++--
 .../flink/table/api/internal/TableEnvironmentImpl.java   | 11 +++++------
 .../org/apache/flink/table/catalog/FunctionCatalog.java  |  6 ------
 .../table/expressions/resolver/ExpressionResolver.java   |  2 +-
 .../apache/flink/table/utils/TableEnvironmentMock.java   |  3 +--
 .../scala/internal/StreamTableEnvironmentImpl.scala      |  2 +-
 .../table/planner/plan/QueryOperationConverter.java      |  4 +---
 .../plan/abilities/source/WatermarkPushDownSpec.java     |  5 +----
 .../planner/plan/nodes/exec/serde/SerdeContext.java      |  6 +++---
 .../plan/nodes/exec/spec/DynamicTableSinkSpec.java       |  2 +-
 .../plan/nodes/exec/spec/DynamicTableSourceSpec.java     |  2 +-
 .../plan/nodes/exec/spec/DynamicTableSpecBase.java       |  5 +----
 .../table/planner/plan/schema/CatalogSourceTable.java    |  6 ++----
 .../planner/utils/DummyStreamExecutionEnvironment.java   |  6 ++++++
 .../apache/flink/table/planner/utils/ShortcutUtils.java  |  2 +-
 .../flink/table/planner/delegation/PlannerBase.scala     | 16 +++++++---------
 .../flink/table/planner/delegation/StreamPlanner.scala   |  1 +
 .../optimize/BatchCommonSubGraphBasedOptimizer.scala     |  2 +-
 .../flink/table/planner/plan/utils/PartitionPruner.scala |  7 +------
 .../plan/nodes/exec/serde/LookupKeySerdeTest.java        |  2 +-
 .../generated/GeneratedWatermarkGeneratorSupplier.java   |  9 +++------
 28 files changed, 57 insertions(+), 78 deletions(-)

diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java
index 51659cd..90e3bc7 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDynamicTableFactoryTest.java
@@ -290,7 +290,7 @@ public class HiveDynamicTableFactoryTest {
                         hiveCatalog.getFactory().orElseThrow(IllegalStateException::new),
                 tableIdentifier,
                 tableEnvInternal.getCatalogManager().resolveCatalogTable(catalogTable),
-                tableEnv.getConfig().getConfiguration(),
+                tableEnv.getConfig(),
                 Thread.currentThread().getContextClassLoader(),
                 false);
     }
@@ -306,7 +306,7 @@ public class HiveDynamicTableFactoryTest {
                         hiveCatalog.getFactory().orElseThrow(IllegalStateException::new),
                 tableIdentifier,
                 tableEnvInternal.getCatalogManager().resolveCatalogTable(catalogTable),
-                tableEnv.getConfig().getConfiguration(),
+                tableEnv.getConfig(),
                 Thread.currentThread().getContextClassLoader(),
                 false);
     }
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java
index 1dbb9a4..38834a1 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveLookupJoinITCase.java
@@ -408,7 +408,7 @@ public class HiveLookupJoinITCase {
                                 tableEnvInternal
                                         .getCatalogManager()
                                         .resolveCatalogTable(catalogTable),
-                                tableEnv.getConfig().getConfiguration(),
+                                tableEnv.getConfig(),
                                 Thread.currentThread().getContextClassLoader(),
                                 false);
         FileSystemLookupFunction<HiveTablePartition> lookupFunction =
diff --git a/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java b/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java
index 158baba..e7c4d8a 100644
--- a/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java
+++ b/flink-python/src/main/java/org/apache/flink/client/python/PythonFunctionFactory.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.java.ExecutionEnvironment;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.python.util.PythonDependencyUtils;
+import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.functions.python.PythonFunction;
 import org.apache.flink.util.FileUtils;
 import org.apache.flink.util.concurrent.ExecutorThreadFactory;
@@ -124,7 +125,11 @@ public interface PythonFunctionFactory {
         Configuration mergedConfig =
                 new Configuration(
                         ExecutionEnvironment.getExecutionEnvironment().getConfiguration());
-        PythonDependencyUtils.merge(mergedConfig, (Configuration) config);
+        if (config instanceof TableConfig) {
+            PythonDependencyUtils.merge(mergedConfig, ((TableConfig) config).getConfiguration());
+        } else {
+            PythonDependencyUtils.merge(mergedConfig, (Configuration) config);
+        }
         PythonFunctionFactory pythonFunctionFactory =
                 PYTHON_FUNCTION_FACTORY_CACHE.get(CacheKey.of(mergedConfig, classLoader));
         ensureCacheCleanupExecutorServiceStarted();
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java
index a72fdf7..ddc1f5f 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/CliResultViewTest.java
@@ -21,7 +21,6 @@ import org.apache.flink.api.common.RuntimeExecutionMode;
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.internal.TableResultInternal;
 import org.apache.flink.table.catalog.Column;
 import org.apache.flink.table.catalog.ResolvedSchema;
@@ -139,8 +138,7 @@ public class CliResultViewTest {
 
         private final TypedResult<?> typedResult;
         private final CountDownLatch cancellationCounter;
-        private static final Configuration defaultConfig =
-                TableConfig.getDefault().getConfiguration();
+        private static final Configuration defaultConfig = new Configuration();
 
         public MockExecutor(TypedResult<?> typedResult, CountDownLatch cancellationCounter) {
             this.typedResult = typedResult;
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java
index 759db8f..fd8fb09 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/cli/TestingExecutor.java
@@ -19,7 +19,6 @@ package org.apache.flink.table.client.cli;
 
 import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.internal.TableResultInternal;
 import org.apache.flink.table.client.cli.utils.SqlParserHelper;
 import org.apache.flink.table.client.gateway.Executor;
@@ -40,7 +39,7 @@ import java.util.Map;
 /** A customizable {@link Executor} for testing purposes. */
 class TestingExecutor implements Executor {
 
-    private static final Configuration defaultConfig = TableConfig.getDefault().getConfiguration();
+    private static final Configuration defaultConfig = new Configuration();
     private int numCancelCalls = 0;
 
     private int numRetrieveResultChancesCalls = 0;
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
index 2d93850..8078fab 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/context/SessionContextTest.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.client.gateway.context;
 
 import org.apache.flink.client.cli.DefaultCLI;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.table.client.gateway.utils.UserDefinedFunctions;
 import org.apache.flink.table.utils.TestUserClassLoaderJar;
 
@@ -222,12 +223,8 @@ public class SessionContextTest {
                 .toMap();
     }
 
-    private Configuration getConfiguration() {
-        return sessionContext
-                .getExecutionContext()
-                .getTableEnvironment()
-                .getConfig()
-                .getConfiguration();
+    private ReadableConfig getConfiguration() {
+        return sessionContext.getExecutionContext().getTableEnvironment().getConfig();
     }
 
     private void validateAddJar(String jarPath) throws IOException {
diff --git a/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java b/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java
index c8df950..16c52b7 100644
--- a/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java
+++ b/flink-table/flink-table-api-bridge-base/src/main/java/org/apache/flink/table/api/bridge/internal/AbstractStreamTableEnvironmentImpl.java
@@ -225,7 +225,8 @@ public abstract class AbstractStreamTableEnvironmentImpl extends TableEnvironmen
         final Transformation<T> transformation = getTransformation(table, transformations);
         executionEnvironment.addOperator(transformation);
 
-        // reconfigure whenever planner transformations are added
+        // Reconfigure whenever planner transformations are added
+        // We pass only the configuration to avoid reconfiguration with the rootConfiguration
         executionEnvironment.configure(tableConfig.getConfiguration());
 
         return new DataStream<>(executionEnvironment, transformation);
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java
index bf70126..bb0e0ac 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableResult.java
@@ -178,7 +178,7 @@ public interface TableResult {
      *
      * <p>This method has slightly different behaviors under different checkpointing settings (to
      * enable checkpointing for a streaming job, set checkpointing properties through {@link
-     * TableConfig#getConfiguration()}).
+     * TableConfig}).
      *
      * <ul>
      *   <li>For batch jobs or streaming jobs without checkpointing, this method has neither
@@ -205,7 +205,7 @@ public interface TableResult {
      *
      * <p>This method has slightly different behaviors under different checkpointing settings (to
      * enable checkpointing for a streaming job, set checkpointing properties through {@link
-     * TableConfig#getConfiguration()}).
+     * TableConfig}).
      *
      * <ul>
      *   <li>For batch jobs or streaming jobs without checkpointing, this method has neither
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
index 91207f5..0d5e8c6 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
@@ -287,7 +287,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
         final CatalogManager catalogManager =
                 CatalogManager.newBuilder()
                         .classLoader(classLoader)
-                        .config(tableConfig.getConfiguration())
+                        .config(tableConfig)
                         .defaultCatalog(
                                 settings.getBuiltInCatalogName(),
                                 new GenericInMemoryCatalog(
@@ -796,6 +796,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
     private TableResultInternal executeInternal(
             List<Transformation<?>> transformations, List<String> sinkIdentifierNames) {
         final String defaultJobName = "insert-into_" + String.join(",", sinkIdentifierNames);
+        // We pass only the configuration to avoid reconfiguration with the rootConfiguration
         Pipeline pipeline =
                 execEnv.createPipeline(
                         transformations, tableConfig.getConfiguration(), defaultJobName);
@@ -826,6 +827,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
         List<Transformation<?>> transformations =
                 translate(Collections.singletonList(sinkOperation));
         final String defaultJobName = "collect";
+        // We pass only the configuration to avoid reconfiguration with the rootConfiguration
         Pipeline pipeline =
                 execEnv.createPipeline(
                         transformations, tableConfig.getConfiguration(), defaultJobName);
@@ -1352,10 +1354,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
 
             Catalog catalog =
                     FactoryUtil.createCatalog(
-                            catalogName,
-                            properties,
-                            tableConfig.getConfiguration(),
-                            userClassLoader);
+                            catalogName, properties, tableConfig, userClassLoader);
             catalogManager.registerCatalog(catalogName, catalog);
 
             return TableResultImpl.TABLE_RESULT_OK;
@@ -1371,7 +1370,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
                     FactoryUtil.createModule(
                             operation.getModuleName(),
                             operation.getOptions(),
-                            tableConfig.getConfiguration(),
+                            tableConfig,
                             userClassLoader);
             moduleManager.loadModule(operation.getModuleName(), module);
             return TableResultImpl.TABLE_RESULT_OK;
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java
index afa626b..6b11ec9 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/catalog/FunctionCatalog.java
@@ -21,7 +21,6 @@ package org.apache.flink.table.catalog;
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.api.common.typeinfo.TypeInformation;
 import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
@@ -77,11 +76,6 @@ public final class FunctionCatalog {
     private PlannerTypeInferenceUtil plannerTypeInferenceUtil;
 
     public FunctionCatalog(
-            TableConfig tableConfig, CatalogManager catalogManager, ModuleManager moduleManager) {
-        this(checkNotNull(tableConfig).getConfiguration(), catalogManager, moduleManager);
-    }
-
-    public FunctionCatalog(
             ReadableConfig config, CatalogManager catalogManager, ModuleManager moduleManager) {
         this.config = checkNotNull(config);
         this.catalogManager = checkNotNull(catalogManager);
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java
index 281866f..ea6f8e3 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/expressions/resolver/ExpressionResolver.java
@@ -137,7 +137,7 @@ public class ExpressionResolver {
             List<LocalReferenceExpression> localReferences,
             @Nullable DataType outputDataType,
             boolean isGroupedAggregation) {
-        this.config = Preconditions.checkNotNull(tableConfig).getConfiguration();
+        this.config = Preconditions.checkNotNull(tableConfig);
         this.tableLookup = Preconditions.checkNotNull(tableLookup);
         this.fieldLookup = Preconditions.checkNotNull(fieldLookup);
         this.functionLookup = Preconditions.checkNotNull(functionLookup);
diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java
index f917132..9403063 100644
--- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java
+++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/utils/TableEnvironmentMock.java
@@ -78,8 +78,7 @@ public class TableEnvironmentMock extends TableEnvironmentImpl {
                 moduleManager,
                 tableConfig,
                 createExecutor(),
-                createFunctionCatalog(
-                        tableConfig.getConfiguration(), catalogManager, moduleManager),
+                createFunctionCatalog(tableConfig, catalogManager, moduleManager),
                 createPlanner(),
                 isStreamingMode);
     }
diff --git a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala
index cbc4f38..1c22f5d 100644
--- a/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala
+++ b/flink-table/flink-table-api-scala-bridge/src/main/scala/org/apache/flink/table/api/bridge/scala/internal/StreamTableEnvironmentImpl.scala
@@ -312,7 +312,7 @@ object StreamTableEnvironmentImpl {
 
     val catalogManager = CatalogManager.newBuilder
       .classLoader(classLoader)
-      .config(tableConfig.getConfiguration)
+      .config(tableConfig)
       .defaultCatalog(
         settings.getBuiltInCatalogName,
         new GenericInMemoryCatalog(
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java
index e688f23..837b3b2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/QueryOperationConverter.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.planner.plan;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.api.TableSchema;
@@ -527,10 +526,9 @@ public class QueryOperationConverter extends QueryOperationDefaultVisitor<RelNod
                 boolean isTopLevelRecord,
                 ChangelogMode changelogMode) {
             final FlinkContext flinkContext = ShortcutUtils.unwrapContext(relBuilder);
-            final ReadableConfig config = flinkContext.getTableConfig().getConfiguration();
             return DynamicSourceUtils.convertDataStreamToRel(
                     flinkContext.isBatchMode(),
-                    config,
+                    flinkContext.getTableConfig(),
                     relBuilder,
                     contextResolvedTable,
                     dataStream,
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java
index 59ff267..7fcb6d7 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/abilities/source/WatermarkPushDownSpec.java
@@ -20,7 +20,6 @@ package org.apache.flink.table.planner.plan.abilities.source;
 
 import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier;
 import org.apache.flink.api.common.eventtime.WatermarkStrategy;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.api.TableException;
 import org.apache.flink.table.connector.source.DynamicTableSource;
 import org.apache.flink.table.connector.source.abilities.SupportsWatermarkPushDown;
@@ -79,11 +78,9 @@ public final class WatermarkPushDownSpec extends SourceAbilitySpecBase {
                             context.getSourceRowType(),
                             watermarkExpr,
                             Option.apply("context"));
-            Configuration configuration = context.getTableConfig().getConfiguration();
 
             WatermarkGeneratorSupplier<RowData> supplier =
-                    new GeneratedWatermarkGeneratorSupplier(
-                            configuration, generatedWatermarkGenerator);
+                    new GeneratedWatermarkGeneratorSupplier(generatedWatermarkGenerator);
 
             WatermarkStrategy<RowData> watermarkStrategy = WatermarkStrategy.forGenerator(supplier);
             if (idleTimeoutMillis > 0) {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java
index 1274132..f02afde 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/SerdeContext.java
@@ -19,7 +19,7 @@
 package org.apache.flink.table.planner.plan.nodes.exec.serde;
 
 import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.table.delegation.Parser;
 import org.apache.flink.table.planner.calcite.FlinkContext;
 import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
@@ -71,8 +71,8 @@ public class SerdeContext {
         return parser;
     }
 
-    public Configuration getConfiguration() {
-        return flinkContext.getTableConfig().getConfiguration();
+    public ReadableConfig getConfiguration() {
+        return flinkContext.getTableConfig();
     }
 
     public ClassLoader getClassLoader() {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java
index 73cb36c..4f092a2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSinkSpec.java
@@ -87,7 +87,7 @@ public class DynamicTableSinkSpec extends DynamicTableSpecBase {
                             contextResolvedTable.getIdentifier(),
                             contextResolvedTable.getResolvedTable(),
                             loadOptionsFromCatalogTable(contextResolvedTable, flinkContext),
-                            flinkContext.getTableConfig().getConfiguration(),
+                            flinkContext.getTableConfig(),
                             flinkContext.getClassLoader(),
                             contextResolvedTable.isTemporary());
             if (sinkAbilities != null) {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java
index b0fd9ce..3cabdb9 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSourceSpec.java
@@ -80,7 +80,7 @@ public class DynamicTableSourceSpec extends DynamicTableSpecBase {
                             contextResolvedTable.getIdentifier(),
                             contextResolvedTable.getResolvedTable(),
                             loadOptionsFromCatalogTable(contextResolvedTable, flinkContext),
-                            flinkContext.getTableConfig().getConfiguration(),
+                            flinkContext.getTableConfig(),
                             flinkContext.getClassLoader(),
                             contextResolvedTable.isTemporary());
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java
index b8d941f..3521dbf 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DynamicTableSpecBase.java
@@ -38,10 +38,7 @@ class DynamicTableSpecBase {
         // In case of CatalogPlanRestore.IDENTIFIER, getCatalogTable() already returns the table
         //  loaded from the catalog
         final TableConfigOptions.CatalogPlanRestore catalogPlanRestore =
-                flinkContext
-                        .getTableConfig()
-                        .getConfiguration()
-                        .get(TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS);
+                flinkContext.getTableConfig().get(TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS);
         if (!contextResolvedTable.isPermanent()
                 || catalogPlanRestore != TableConfigOptions.CatalogPlanRestore.ALL) {
             return Collections.emptyMap();
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java
index 14fe596..385d930 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/schema/CatalogSourceTable.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.table.planner.plan.schema;
 
-import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.catalog.Catalog;
@@ -118,7 +117,7 @@ public final class CatalogSourceTable extends FlinkPreparingTableBase {
         // prepare table source and convert to RelNode
         return DynamicSourceUtils.convertSourceToRel(
                 !schemaTable.isStreamingMode(),
-                context.getTableConfig().getConfiguration(),
+                context.getTableConfig(),
                 relBuilder,
                 schemaTable.getContextResolvedTable(),
                 schemaTable.getStatistic(),
@@ -153,7 +152,6 @@ public final class CatalogSourceTable extends FlinkPreparingTableBase {
 
     private DynamicTableSource createDynamicTableSource(
             FlinkContext context, ResolvedCatalogTable catalogTable) {
-        final ReadableConfig config = context.getTableConfig().getConfiguration();
 
         final Optional<DynamicTableSourceFactory> factoryFromCatalog =
                 schemaTable
@@ -178,7 +176,7 @@ public final class CatalogSourceTable extends FlinkPreparingTableBase {
                 factory,
                 schemaTable.getContextResolvedTable().getIdentifier(),
                 catalogTable,
-                config,
+                context.getTableConfig(),
                 Thread.currentThread().getContextClassLoader(),
                 schemaTable.isTemporary());
     }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java
index e3db533..02faf81 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/DummyStreamExecutionEnvironment.java
@@ -24,6 +24,7 @@ import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.api.common.restartstrategy.RestartStrategies;
 import org.apache.flink.api.dag.Transformation;
 import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.runtime.state.StateBackend;
 import org.apache.flink.streaming.api.CheckpointingMode;
 import org.apache.flink.streaming.api.TimeCharacteristic;
@@ -85,6 +86,11 @@ public class DummyStreamExecutionEnvironment extends StreamExecutionEnvironment
     }
 
     @Override
+    public ReadableConfig getConfiguration() {
+        return realExecEnv.getConfiguration();
+    }
+
+    @Override
     public List<Tuple2<String, DistributedCache.DistributedCacheEntry>> getCachedFiles() {
         return realExecEnv.getCachedFiles();
     }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java
index 90914da7..7cc41f1 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java
@@ -89,7 +89,7 @@ public final class ShortcutUtils {
     }
 
     public static ReadableConfig unwrapTableConfig(RelNode relNode) {
-        return unwrapContext(relNode).getTableConfig().getConfiguration();
+        return unwrapContext(relNode).getTableConfig();
     }
 
     public static @Nullable FunctionDefinition unwrapFunctionDefinition(
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
index b31606e..05d7baf 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
@@ -226,7 +226,7 @@ abstract class PlannerBase(
           getRelBuilder,
           input,
           collectModifyOperation,
-          getTableConfig.getConfiguration,
+          getTableConfig,
           getFlinkContext.getClassLoader
         )
 
@@ -402,7 +402,7 @@ abstract class PlannerBase(
             catalog.orNull,
             objectIdentifier,
             tableToFind.getOrigin,
-            getTableConfig.getConfiguration,
+            getTableConfig,
             isStreamingMode,
             isTemporary)
           Option(resolvedTable, tableSink)
@@ -424,7 +424,7 @@ abstract class PlannerBase(
             objectIdentifier,
             tableToFind,
             Collections.emptyMap(),
-            getTableConfig.getConfiguration,
+            getTableConfig,
             getFlinkContext.getClassLoader,
             isTemporary)
           Option(resolvedTable, tableSink)
@@ -455,7 +455,7 @@ abstract class PlannerBase(
           catalog.orElse(null),
           objectIdentifier,
           catalogTable,
-          getTableConfig.getConfiguration,
+          getTableConfig,
           isStreamingMode,
           isTemporary)
         // success, then we will use the legacy factories
@@ -479,8 +479,6 @@ abstract class PlannerBase(
   }
 
   protected def beforeTranslation(): Unit = {
-    val configuration = tableConfig.getConfiguration
-
     // Add query start time to TableConfig, these config are used internally,
     // these configs will be used by temporal functions like CURRENT_TIMESTAMP,LOCALTIMESTAMP.
     val epochTime :JLong = System.currentTimeMillis()
@@ -489,9 +487,8 @@ abstract class PlannerBase(
       TimeZone.getTimeZone(tableConfig.getLocalTimeZone).getOffset(epochTime)
     tableConfig.set(TABLE_QUERY_START_LOCAL_TIME, localTime)
 
-    getExecEnv.configure(
-      configuration,
-      Thread.currentThread().getContextClassLoader)
+    // We pass only the configuration to avoid reconfiguration with the rootConfiguration
+    getExecEnv.configure(tableConfig.getConfiguration, Thread.currentThread().getContextClassLoader)
 
     // Use config parallelism to override env parallelism.
     val defaultParallelism = getTableConfig.get(
@@ -548,6 +545,7 @@ abstract class PlannerBase(
     val transformations = translateToPlan(execGraph)
     afterTranslation()
 
+    // We pass only the configuration to avoid reconfiguration with the rootConfiguration
     val streamGraph = executor.createPipeline(transformations, tableConfig.getConfiguration, null)
       .asInstanceOf[StreamGraph]
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala
index 6e896d8..c3839bc 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala
@@ -190,6 +190,7 @@ class StreamPlanner(
     val transformations = translateToPlan(execGraph)
     afterTranslation()
 
+    // We pass only the configuration to avoid reconfiguration with the rootConfiguration
     val streamGraph = executor.createPipeline(transformations, tableConfig.getConfiguration, null)
       .asInstanceOf[StreamGraph]
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
index 5e3c42b..62d2d9a 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
@@ -79,7 +79,7 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner)
   private def optimizeTree(relNode: RelNode): RelNode = {
     val config = planner.getTableConfig
     val programs = TableConfigUtils.getCalciteConfig(config).getBatchProgram
-      .getOrElse(FlinkBatchProgram.buildProgram(config.getConfiguration))
+      .getOrElse(FlinkBatchProgram.buildProgram(config))
     Preconditions.checkNotNull(programs)
 
     val context = relNode.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext])
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala
index 9f1c47b..49d05b3 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala
@@ -119,13 +119,8 @@ object PartitionPruner {
     val results: JList[Boolean] = new JArrayList[Boolean](allPartitions.size)
     val collector = new ListCollector[Boolean](results)
 
-    val parameters = if (tableConfig.getConfiguration != null) {
-      tableConfig.getConfiguration
-    } else {
-      new Configuration()
-    }
     try {
-      richMapFunction.open(parameters)
+      richMapFunction.open(new Configuration)
       // do filter against all partitions
       allPartitions.foreach { partition =>
         val row = convertPartitionToRow(
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java
index 2c3f372..a1f48c9 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LookupKeySerdeTest.java
@@ -50,7 +50,7 @@ public class LookupKeySerdeTest {
         CatalogManager catalogManager =
                 CatalogManager.newBuilder()
                         .classLoader(Thread.currentThread().getContextClassLoader())
-                        .config(tableConfig.getConfiguration())
+                        .config(tableConfig)
                         .defaultCatalog("default_catalog", new GenericInMemoryCatalog("default_db"))
                         .build();
         FlinkContext flinkContext =
diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java
index 583b3c2..11fdc4a 100644
--- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java
+++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java
@@ -39,12 +39,10 @@ import java.util.List;
 public class GeneratedWatermarkGeneratorSupplier implements WatermarkGeneratorSupplier<RowData> {
     private static final long serialVersionUID = 1L;
 
-    private final Configuration configuration;
     private final GeneratedWatermarkGenerator generatedWatermarkGenerator;
 
     public GeneratedWatermarkGeneratorSupplier(
-            Configuration configuration, GeneratedWatermarkGenerator generatedWatermarkGenerator) {
-        this.configuration = configuration;
+            GeneratedWatermarkGenerator generatedWatermarkGenerator) {
         this.generatedWatermarkGenerator = generatedWatermarkGenerator;
     }
 
@@ -60,12 +58,11 @@ public class GeneratedWatermarkGeneratorSupplier implements WatermarkGeneratorSu
                 new GeneratedWatermarkGenerator(
                                 generatedWatermarkGenerator.getClassName(),
                                 generatedWatermarkGenerator.getCode(),
-                                references.toArray(),
-                                configuration)
+                                references.toArray())
                         .newInstance(Thread.currentThread().getContextClassLoader());
 
         try {
-            innerWatermarkGenerator.open(configuration);
+            innerWatermarkGenerator.open(new Configuration());
         } catch (Exception e) {
             throw new RuntimeException("Fail to instantiate generated watermark generator.", e);
         }

[flink] 02/08: [FLINK-26709][table] Replace TableConfig.getConfiguration.set()

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 5bd6aeffd0e0aa3bdde155e2a6a84c091de79790
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 16:10:30 2022 +0200

    [FLINK-26709][table] Replace TableConfig.getConfiguration.set()
    
    Since `TableConfig` is a `WritableConfig`, callers should directly call
    `TableConfig.set()` and avoid going throught `#getConfiguration()` which
    is there only for advanced internal configuration read value purposes.
    
    (cherry picked from commit 79110e987484a63184a5b978aea53ff402ba4c57)
---
 .../table/ElasticsearchDynamicSinkBaseITCase.java  |   6 +-
 .../flink/connectors/hive/HiveRunnerITCase.java    |   4 +-
 .../connectors/hive/HiveSinkCompactionITCase.java  |   4 +-
 .../flink/connectors/hive/HiveTableSinkITCase.java |   8 +-
 .../connectors/hive/HiveTableSourceITCase.java     |  31 +---
 .../hive/TableEnvHiveConnectorITCase.java          |   8 +-
 .../table/catalog/hive/HiveCatalogITCase.java      |  14 +-
 .../flink/table/catalog/hive/HiveTestUtils.java    |   8 +-
 .../connector/jdbc/catalog/MySqlCatalogITCase.java |   4 +-
 .../jdbc/catalog/PostgresCatalogITCase.java        |   4 +-
 .../kafka/table/KafkaChangelogTableITCase.java     |  37 +++--
 .../connectors/kafka/table/KafkaTableITCase.java   |  11 +-
 .../flink/python/tests/BatchPythonUdfSqlJob.java   |   2 +-
 .../apache/flink/table/tpcds/TpcdsTestProgram.java |  12 +-
 .../client/python/PythonFunctionFactoryTest.java   |   5 +-
 .../PythonScalarFunctionOperatorTestBase.java      |   5 +-
 .../table/planner/delegation/PlannerBase.scala     |   4 +-
 .../BuiltInAggregateFunctionTestBase.java          |   4 +-
 .../nodes/exec/serde/LogicalTypeJsonSerdeTest.java |  16 +-
 .../PushLocalAggIntoTableSourceScanRuleTest.java   |   3 +-
 .../plan/batch/sql/DeadlockBreakupTest.scala       | 110 +++++++-------
 .../planner/plan/batch/sql/LegacySinkTest.scala    |   5 +-
 .../plan/batch/sql/MultipleInputCreationTest.scala |  56 +++----
 .../plan/batch/sql/RemoveCollationTest.scala       |  40 ++---
 .../planner/plan/batch/sql/RemoveShuffleTest.scala | 168 ++++++++++-----------
 .../planner/plan/batch/sql/SetOperatorsTest.scala  |   2 +-
 .../planner/plan/batch/sql/SortLimitTest.scala     |  24 +--
 .../table/planner/plan/batch/sql/SortTest.scala    |  34 ++---
 .../planner/plan/batch/sql/SubplanReuseTest.scala  |  68 ++++-----
 .../planner/plan/batch/sql/TableSinkTest.scala     |   5 +-
 .../planner/plan/batch/sql/TableSourceTest.scala   |   5 +-
 .../plan/batch/sql/agg/GroupWindowTest.scala       |   2 +-
 .../plan/batch/sql/agg/HashAggregateTest.scala     |   4 +-
 .../plan/batch/sql/agg/SortAggregateTest.scala     |   4 +-
 .../batch/sql/join/BroadcastHashJoinTest.scala     |   7 +-
 .../sql/join/BroadcastHashSemiAntiJoinTest.scala   |   6 +-
 .../plan/batch/sql/join/LookupJoinTest.scala       |   4 +-
 .../plan/batch/sql/join/NestedLoopJoinTest.scala   |   2 +-
 .../sql/join/NestedLoopSemiAntiJoinTest.scala      |   2 +-
 .../plan/batch/sql/join/ShuffledHashJoinTest.scala |   2 +-
 .../sql/join/ShuffledHashSemiAntiJoinTest.scala    |   2 +-
 .../plan/batch/sql/join/SortMergeJoinTest.scala    |   2 +-
 .../batch/sql/join/SortMergeSemiAntiJoinTest.scala |   2 +-
 .../planner/plan/common/JoinReorderTestBase.scala  |   8 +-
 .../planner/plan/common/TableFactoryTest.scala     |   2 +-
 .../table/planner/plan/hint/OptionsHintTest.scala  |   4 +-
 .../logical/JoinDeriveNullFilterRuleTest.scala     |   4 +-
 .../rules/logical/SplitAggregateRuleTest.scala     |  16 +-
 .../batch/EnforceLocalHashAggRuleTest.scala        |   4 +-
 .../batch/EnforceLocalSortAggRuleTest.scala        |   4 +-
 .../RemoveRedundantLocalHashAggRuleTest.scala      |  18 +--
 .../stream/ChangelogModeInferenceTest.scala        |   8 +-
 ...xpandWindowTableFunctionTransposeRuleTest.scala |   4 +-
 .../plan/stream/sql/DagOptimizationTest.scala      |  55 +++----
 .../planner/plan/stream/sql/DeduplicateTest.scala  |  14 +-
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  42 +++---
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  18 +--
 .../table/planner/plan/stream/sql/RankTest.scala   |   5 +-
 .../planner/plan/stream/sql/SubplanReuseTest.scala |  34 ++---
 .../planner/plan/stream/sql/TableScanTest.scala    |  12 +-
 .../planner/plan/stream/sql/TableSinkTest.scala    |  23 ++-
 .../plan/stream/sql/agg/AggregateTest.scala        |  24 +--
 .../stream/sql/agg/DistinctAggregateTest.scala     |  11 +-
 .../plan/stream/sql/agg/GroupWindowTest.scala      |  16 +-
 .../stream/sql/agg/IncrementalAggregateTest.scala  |   4 +-
 .../stream/sql/agg/TwoStageAggregateTest.scala     |   2 +-
 .../plan/stream/sql/agg/WindowAggregateTest.scala  |  38 ++---
 .../utils/StreamingWithMiniBatchTestBase.scala     |   6 +-
 68 files changed, 543 insertions(+), 579 deletions(-)

diff --git a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java
index 21ad5fd..75dd9b9 100644
--- a/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java
+++ b/flink-connectors/flink-connector-elasticsearch-base/src/test/java/org/apache/flink/connector/elasticsearch/table/ElasticsearchDynamicSinkBaseITCase.java
@@ -25,6 +25,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.catalog.Column;
 import org.apache.flink.table.catalog.ResolvedSchema;
 import org.apache.flink.table.catalog.UniqueConstraint;
@@ -310,10 +311,7 @@ abstract class ElasticsearchDynamicSinkBaseITCase {
                 TableEnvironment.create(EnvironmentSettings.inStreamingMode());
 
         DateTimeFormatter dateTimeFormatter = DateTimeFormatter.ofPattern("yyyy-MM-dd");
-        tableEnvironment
-                .getConfig()
-                .getConfiguration()
-                .setString("table.local-time-zone", "Asia/Shanghai");
+        tableEnvironment.getConfig().set(TableConfigOptions.LOCAL_TIME_ZONE, "Asia/Shanghai");
 
         String dynamicIndex1 =
                 "dynamic-index-"
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java
index 2b2d793..606c351 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveRunnerITCase.java
@@ -539,9 +539,7 @@ public class HiveRunnerITCase {
             tableEnv.executeSql("create table db1.src (x smallint,y int) stored as orc");
             hiveShell.execute("insert into table db1.src values (1,100),(2,200)");
 
-            tableEnv.getConfig()
-                    .getConfiguration()
-                    .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true);
+            tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true);
 
             tableEnv.executeSql("alter table db1.src change x x int");
             assertEquals(
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java
index d2f6db5..ffe2222 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveSinkCompactionITCase.java
@@ -55,9 +55,7 @@ public class HiveSinkCompactionITCase extends CompactionITCaseBase {
         tEnv().useCatalog(hiveCatalog.getName());
 
         // avoid too large parallelism lead to scheduler dead lock in streaming mode
-        tEnv().getConfig()
-                .getConfiguration()
-                .set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false);
+        tEnv().getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false);
 
         super.init();
     }
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
index 1d2b4f5..39f7950 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkITCase.java
@@ -569,13 +569,9 @@ public class HiveTableSinkITCase {
         tEnv.useCatalog(hiveCatalog.getName());
         tEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
         if (useMr) {
-            tEnv.getConfig()
-                    .getConfiguration()
-                    .set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, true);
+            tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, true);
         } else {
-            tEnv.getConfig()
-                    .getConfiguration()
-                    .set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, false);
+            tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_WRITER, false);
         }
 
         try {
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java
index d9e1fad..df3023f 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceITCase.java
@@ -473,9 +473,7 @@ public class HiveTableSourceITCase extends BatchAbstractTestBase {
                         "select * from hive.source_db.test_parallelism_setting_with_file_num");
         testParallelismSettingTranslateAndAssert(3, table, tEnv);
 
-        tEnv.getConfig()
-                .getConfiguration()
-                .setInteger(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MAX, 2);
+        tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM_MAX, 2);
         testParallelismSettingTranslateAndAssert(2, table, tEnv);
     }
 
@@ -496,12 +494,8 @@ public class HiveTableSourceITCase extends BatchAbstractTestBase {
         final String dbName = "source_db";
         final String tblName = "test_parallelism_limit_pushdown";
         TableEnvironment tEnv = createTableEnv();
-        tEnv.getConfig()
-                .getConfiguration()
-                .setBoolean(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false);
-        tEnv.getConfig()
-                .getConfiguration()
-                .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2);
+        tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false);
+        tEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2);
         tEnv.executeSql(
                 "CREATE TABLE source_db.test_parallelism_limit_pushdown "
                         + "(`year` STRING, `value` INT) partitioned by (pt int)");
@@ -536,9 +530,7 @@ public class HiveTableSourceITCase extends BatchAbstractTestBase {
         tEnv.getConfig().setSqlDialect(SqlDialect.HIVE);
         tEnv.registerCatalog("hive", hiveCatalog);
         tEnv.useCatalog("hive");
-        tEnv.getConfig()
-                .getConfiguration()
-                .setBoolean(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false);
+        tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, false);
         tEnv.executeSql(
                 "CREATE TABLE source_db.test_parallelism_no_infer "
                         + "(`year` STRING, `value` INT) partitioned by (pt int)");
@@ -773,9 +765,7 @@ public class HiveTableSourceITCase extends BatchAbstractTestBase {
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         StreamTableEnvironment tEnv =
                 HiveTestUtils.createTableEnvInStreamingMode(env, SqlDialect.HIVE);
-        tEnv.getConfig()
-                .getConfiguration()
-                .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, useMapredReader);
+        tEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, useMapredReader);
         tEnv.registerCatalog(catalogName, hiveCatalog);
         tEnv.useCatalog(catalogName);
         tEnv.executeSql(
@@ -838,15 +828,10 @@ public class HiveTableSourceITCase extends BatchAbstractTestBase {
         doReturn(Optional.of(tableFactorySpy)).when(catalogSpy).getTableFactory();
 
         TableEnvironment tableEnv = HiveTestUtils.createTableEnvInBatchMode();
+        tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, fallbackMR);
         tableEnv.getConfig()
-                .getConfiguration()
-                .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, fallbackMR);
-        tableEnv.getConfig()
-                .getConfiguration()
-                .setBoolean(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, inferParallelism);
-        tableEnv.getConfig()
-                .getConfiguration()
-                .setInteger(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2);
+                .set(HiveOptions.TABLE_EXEC_HIVE_INFER_SOURCE_PARALLELISM, inferParallelism);
+        tableEnv.getConfig().set(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 2);
         tableEnv.registerCatalog(catalogSpy.getName(), catalogSpy);
         tableEnv.useCatalog(catalogSpy.getName());
 
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java
index 1d760d0..349264d 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorITCase.java
@@ -431,9 +431,7 @@ public class TableEnvHiveConnectorITCase {
                     String.format(
                             "create table db1.t2 (y int,x int) stored as parquet location '%s'",
                             location));
-            tableEnv.getConfig()
-                    .getConfiguration()
-                    .setBoolean(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true);
+            tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true);
             assertEquals(
                     "[+I[1], +I[2]]",
                     CollectionUtil.iteratorToList(
@@ -584,9 +582,7 @@ public class TableEnvHiveConnectorITCase {
             // test.parquet data: hehuiyuan	{}	[]
             String folderURI = this.getClass().getResource("/parquet").getPath();
 
-            tableEnv.getConfig()
-                    .getConfiguration()
-                    .set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true);
+            tableEnv.getConfig().set(HiveOptions.TABLE_EXEC_HIVE_FALLBACK_MAPRED_READER, true);
             tableEnv.executeSql(
                     String.format(
                             "create external table src_t (a string, b map<string, string>, c array<string>) stored as %s location 'file://%s'",
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java
index b18e11d..77b289e 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java
@@ -218,9 +218,7 @@ public class HiveCatalogITCase {
     public void testReadWriteCsv() throws Exception {
         // similar to CatalogTableITCase::testReadWriteCsvUsingDDL but uses HiveCatalog
         TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
-        tableEnv.getConfig()
-                .getConfiguration()
-                .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
+        tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
 
         tableEnv.registerCatalog("myhive", hiveCatalog);
         tableEnv.useCatalog("myhive");
@@ -304,9 +302,7 @@ public class HiveCatalogITCase {
             settings = EnvironmentSettings.inBatchMode();
         }
         TableEnvironment tableEnv = TableEnvironment.create(settings);
-        tableEnv.getConfig()
-                .getConfiguration()
-                .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
+        tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
 
         tableEnv.registerCatalog("myhive", hiveCatalog);
         tableEnv.useCatalog("myhive");
@@ -335,9 +331,7 @@ public class HiveCatalogITCase {
     @Test
     public void testTableWithPrimaryKey() {
         TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
-        tableEnv.getConfig()
-                .getConfiguration()
-                .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
+        tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
 
         tableEnv.registerCatalog("catalog1", hiveCatalog);
         tableEnv.useCatalog("catalog1");
@@ -386,7 +380,7 @@ public class HiveCatalogITCase {
                 TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build());
         tEnv.registerCatalog("myhive", hiveCatalog);
         tEnv.useCatalog("myhive");
-        tEnv.getConfig().getConfiguration().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
+        tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
 
         String path = this.getClass().getResource("/csv/test.csv").getPath();
 
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java
index b3717ae..4c8c5b7 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveTestUtils.java
@@ -153,9 +153,7 @@ public class HiveTestUtils {
 
     public static TableEnvironment createTableEnvInBatchMode(SqlDialect dialect) {
         TableEnvironment tableEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
-        tableEnv.getConfig()
-                .getConfiguration()
-                .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
+        tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
         tableEnv.getConfig().setSqlDialect(dialect);
         return tableEnv;
     }
@@ -168,9 +166,7 @@ public class HiveTestUtils {
     public static StreamTableEnvironment createTableEnvInStreamingMode(
             StreamExecutionEnvironment env, SqlDialect dialect) {
         StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
-        tableEnv.getConfig()
-                .getConfiguration()
-                .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
+        tableEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
         tableEnv.getConfig().setSqlDialect(dialect);
         return tableEnv;
     }
diff --git a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java
index 9639792..0e2c486 100644
--- a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java
+++ b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java
@@ -155,9 +155,7 @@ public class MySqlCatalogITCase extends MySqlCatalogTestBase {
     @Before
     public void setup() {
         this.tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
-        tEnv.getConfig()
-                .getConfiguration()
-                .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
+        tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
 
         // Use mysql catalog.
         tEnv.registerCatalog(TEST_CATALOG_NAME, catalog);
diff --git a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java
index 936230b..020b7da 100644
--- a/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java
+++ b/flink-connectors/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java
@@ -40,9 +40,7 @@ public class PostgresCatalogITCase extends PostgresCatalogTestBase {
     @Before
     public void setup() {
         this.tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
-        tEnv.getConfig()
-                .getConfiguration()
-                .setInteger(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM.key(), 1);
+        tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
 
         // use PG catalog
         tEnv.registerCatalog(TEST_CATALOG_NAME, catalog);
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java
index 62a7d63..85c4af6 100644
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java
+++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaChangelogTableITCase.java
@@ -20,14 +20,16 @@ package org.apache.flink.streaming.connectors.kafka.table;
 
 import org.apache.flink.api.common.serialization.SerializationSchema;
 import org.apache.flink.api.common.serialization.SimpleStringSchema;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.connector.base.DeliveryGuarantee;
 import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema;
 import org.apache.flink.connector.kafka.sink.KafkaSink;
 import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkFixedPartitioner;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
+import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+import org.apache.flink.table.api.config.OptimizerConfigOptions;
 
 import org.apache.kafka.clients.producer.ProducerConfig;
 import org.junit.Before;
@@ -58,11 +60,12 @@ public class KafkaChangelogTableITCase extends KafkaTableTestBase {
         createTestTopic(topic, 1, 1);
 
         // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
-        Configuration tableConf = tEnv.getConfig().getConfiguration();
-        tableConf.setString("table.exec.mini-batch.enabled", "true");
-        tableConf.setString("table.exec.mini-batch.allow-latency", "1s");
-        tableConf.setString("table.exec.mini-batch.size", "5000");
-        tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE");
+        TableConfig tableConf = tEnv.getConfig();
+        tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true);
+        tableConf.set(
+                ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1));
+        tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L);
+        tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE");
 
         // ---------- Write the Debezium json into Kafka -------------------
         List<String> lines = readLines("debezium-data-schema-exclude.txt");
@@ -186,11 +189,12 @@ public class KafkaChangelogTableITCase extends KafkaTableTestBase {
         // configure time zone of  the Canal Json metadata "ingestion-timestamp"
         tEnv.getConfig().setLocalTimeZone(ZoneId.of("UTC"));
         // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
-        Configuration tableConf = tEnv.getConfig().getConfiguration();
-        tableConf.setString("table.exec.mini-batch.enabled", "true");
-        tableConf.setString("table.exec.mini-batch.allow-latency", "1s");
-        tableConf.setString("table.exec.mini-batch.size", "5000");
-        tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE");
+        TableConfig tableConf = tEnv.getConfig();
+        tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true);
+        tableConf.set(
+                ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1));
+        tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L);
+        tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE");
 
         // ---------- Write the Canal json into Kafka -------------------
         List<String> lines = readLines("canal-data.txt");
@@ -326,11 +330,12 @@ public class KafkaChangelogTableITCase extends KafkaTableTestBase {
         // configure time zone of  the Maxwell Json metadata "ingestion-timestamp"
         tEnv.getConfig().setLocalTimeZone(ZoneId.of("UTC"));
         // enables MiniBatch processing to verify MiniBatch + FLIP-95, see FLINK-18769
-        Configuration tableConf = tEnv.getConfig().getConfiguration();
-        tableConf.setString("table.exec.mini-batch.enabled", "true");
-        tableConf.setString("table.exec.mini-batch.allow-latency", "1s");
-        tableConf.setString("table.exec.mini-batch.size", "5000");
-        tableConf.setString("table.optimizer.agg-phase-strategy", "TWO_PHASE");
+        TableConfig tableConf = tEnv.getConfig();
+        tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true);
+        tableConf.set(
+                ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1));
+        tableConf.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 5000L);
+        tableConf.set(OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE");
 
         // ---------- Write the Maxwell json into Kafka -------------------
         List<String> lines = readLines("maxwell-data.txt");
diff --git a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
index c268453..cfd5a46 100644
--- a/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
+++ b/flink-connectors/flink-connector-kafka/src/test/java/org/apache/flink/streaming/connectors/kafka/table/KafkaTableITCase.java
@@ -24,6 +24,7 @@ import org.apache.flink.streaming.api.datastream.DataStream;
 import org.apache.flink.streaming.api.functions.sink.SinkFunction;
 import org.apache.flink.streaming.connectors.kafka.partitioner.FlinkKafkaPartitioner;
 import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.data.RowData;
 import org.apache.flink.table.utils.EncodingUtils;
 import org.apache.flink.test.util.SuccessException;
@@ -524,7 +525,7 @@ public class KafkaTableITCase extends KafkaTableTestBase {
         // 'value.source.timestamp'` DDL
         // will use the session time zone when convert the changelog time from milliseconds to
         // timestamp
-        tEnv.getConfig().getConfiguration().setString("table.local-time-zone", "UTC");
+        tEnv.getConfig().set(TableConfigOptions.LOCAL_TIME_ZONE, "UTC");
 
         // we always use a different topic name for each parameterized topic,
         // in order to make sure the topic can be created.
@@ -760,9 +761,7 @@ public class KafkaTableITCase extends KafkaTableTestBase {
         // ---------- Produce an event time stream into Kafka -------------------
         String groupId = getStandardProps().getProperty("group.id");
         String bootstraps = getBootstrapServers();
-        tEnv.getConfig()
-                .getConfiguration()
-                .set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100));
+        tEnv.getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100));
 
         final String createTable =
                 String.format(
@@ -878,9 +877,7 @@ public class KafkaTableITCase extends KafkaTableTestBase {
 
         // ---------- Produce an event time stream into Kafka -------------------
         String bootstraps = getBootstrapServers();
-        tEnv.getConfig()
-                .getConfiguration()
-                .set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100));
+        tEnv.getConfig().set(TABLE_EXEC_SOURCE_IDLE_TIMEOUT, Duration.ofMillis(100));
 
         final String createTableSql =
                 "CREATE TABLE %s (\n"
diff --git a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java
index 5639aca..e8d286c 100644
--- a/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java
+++ b/flink-end-to-end-tests/flink-python-test/src/main/java/org/apache/flink/python/tests/BatchPythonUdfSqlJob.java
@@ -32,7 +32,7 @@ public class BatchPythonUdfSqlJob {
 
     public static void main(String[] args) {
         TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
-        tEnv.getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1);
+        tEnv.getConfig().set(CoreOptions.DEFAULT_PARALLELISM, 1);
         tEnv.executeSql(
                 "create temporary system function add_one as 'add_one.add_one' language python");
 
diff --git a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
index c6251aa..7c40ae2 100644
--- a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
+++ b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
@@ -140,18 +140,14 @@ public class TpcdsTestProgram {
         // config Optimizer parameters
         // TODO use the default shuffle mode of batch runtime mode once FLINK-23470 is implemented
         tEnv.getConfig()
-                .getConfiguration()
-                .setString(
+                .set(
                         ExecutionConfigOptions.TABLE_EXEC_SHUFFLE_MODE,
                         GlobalStreamExchangeMode.POINTWISE_EDGES_PIPELINED.toString());
         tEnv.getConfig()
-                .getConfiguration()
-                .setLong(
+                .set(
                         OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD,
-                        10 * 1024 * 1024);
-        tEnv.getConfig()
-                .getConfiguration()
-                .setBoolean(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, true);
+                        10 * 1024 * 1024L);
+        tEnv.getConfig().set(OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, true);
 
         // register TPC-DS tables
         TPCDS_TABLES.forEach(
diff --git a/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java b/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java
index b51cf07..87da19b 100644
--- a/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java
+++ b/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.flink.client.python;
 
+import org.apache.flink.configuration.MemorySize;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.TableEnvironment;
@@ -69,8 +70,8 @@ public class PythonFunctionFactoryTest {
         }
         StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment();
         tableEnv = StreamTableEnvironment.create(sEnv);
-        tableEnv.getConfig().getConfiguration().set(PYTHON_FILES, pyFilePath.getAbsolutePath());
-        tableEnv.getConfig().getConfiguration().setString(TASK_OFF_HEAP_MEMORY.key(), "80mb");
+        tableEnv.getConfig().set(PYTHON_FILES, pyFilePath.getAbsolutePath());
+        tableEnv.getConfig().set(TASK_OFF_HEAP_MEMORY, MemorySize.parse("80mb"));
         sourceTable = tableEnv.fromDataStream(sEnv.fromElements("1", "2", "3")).as("str");
     }
 
diff --git a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java
index 8abfbc6..cc752d3 100644
--- a/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java
+++ b/flink-python/src/test/java/org/apache/flink/table/runtime/operators/python/scalar/PythonScalarFunctionOperatorTestBase.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
 import org.apache.flink.api.common.typeutils.TypeSerializer;
 import org.apache.flink.api.java.tuple.Tuple2;
 import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.MemorySize;
 import org.apache.flink.configuration.TaskManagerOptions;
 import org.apache.flink.core.memory.ManagedMemoryUseCase;
 import org.apache.flink.python.PythonOptions;
@@ -218,9 +219,7 @@ public abstract class PythonScalarFunctionOperatorTestBase<IN, OUT, UDFIN> {
         StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
         env.setParallelism(1);
         StreamTableEnvironment tEnv = createTableEnvironment(env);
-        tEnv.getConfig()
-                .getConfiguration()
-                .setString(TaskManagerOptions.TASK_OFF_HEAP_MEMORY.key(), "80mb");
+        tEnv.getConfig().set(TaskManagerOptions.TASK_OFF_HEAP_MEMORY, MemorySize.parse("80mb"));
         tEnv.registerFunction("pyFunc", new PythonScalarFunction("pyFunc"));
         DataStream<Tuple2<Integer, Integer>> ds = env.fromElements(new Tuple2<>(1, 2));
         Table t = tEnv.fromDataStream(ds, $("a"), $("b")).select(call("pyFunc", $("a"), $("b")));
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
index bf6871c..c4e7aae 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
@@ -484,10 +484,10 @@ abstract class PlannerBase(
     // Add query start time to TableConfig, these config are used internally,
     // these configs will be used by temporal functions like CURRENT_TIMESTAMP,LOCALTIMESTAMP.
     val epochTime :JLong = System.currentTimeMillis()
-    configuration.set(TABLE_QUERY_START_EPOCH_TIME, epochTime)
+    tableConfig.set(TABLE_QUERY_START_EPOCH_TIME, epochTime)
     val localTime :JLong =  epochTime +
       TimeZone.getTimeZone(tableConfig.getLocalTimeZone).getOffset(epochTime)
-    configuration.set(TABLE_QUERY_START_LOCAL_TIME, localTime)
+    tableConfig.set(TABLE_QUERY_START_LOCAL_TIME, localTime)
 
     getExecEnv.configure(
       configuration,
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java
index 352ed83..ba2f01e 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/functions/BuiltInAggregateFunctionTestBase.java
@@ -18,7 +18,6 @@
 
 package org.apache.flink.table.planner.functions;
 
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.StateChangelogOptions;
 import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
 import org.apache.flink.streaming.api.functions.source.SourceFunction;
@@ -79,9 +78,8 @@ public class BuiltInAggregateFunctionTestBase {
     public void testFunction() throws Exception {
         final TableEnvironment tEnv =
                 TableEnvironment.create(EnvironmentSettings.inStreamingMode());
-        Configuration configuration = tEnv.getConfig().getConfiguration();
         // see https://issues.apache.org/jira/browse/FLINK-26092
-        configuration.set(StateChangelogOptions.ENABLE_STATE_CHANGE_LOG, false);
+        tEnv.getConfig().set(StateChangelogOptions.ENABLE_STATE_CHANGE_LOG, false);
         final Table sourceTable = asTable(tEnv, testSpec.sourceRowType, testSpec.sourceRows);
 
         for (final TestItem testItem : testSpec.testItems) {
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java
index 6e29c40..d27dbb5 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/serde/LogicalTypeJsonSerdeTest.java
@@ -19,7 +19,6 @@
 package org.apache.flink.table.planner.plan.nodes.exec.serde;
 
 import org.apache.flink.api.common.typeutils.base.LocalDateTimeSerializer;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.ValidationException;
@@ -112,18 +111,17 @@ public class LogicalTypeJsonSerdeTest {
     public void testIdentifierSerde() throws IOException {
         final DataTypeFactoryMock dataTypeFactoryMock = new DataTypeFactoryMock();
         final TableConfig tableConfig = TableConfig.getDefault();
-        final Configuration config = tableConfig.getConfiguration();
         final CatalogManager catalogManager =
                 preparedCatalogManager().dataTypeFactory(dataTypeFactoryMock).build();
         final SerdeContext serdeContext = configuredSerdeContext(catalogManager, tableConfig);
 
         // minimal plan content
-        config.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, IDENTIFIER);
+        tableConfig.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, IDENTIFIER);
         final String minimalJson = toJson(serdeContext, STRUCTURED_TYPE);
         assertThat(minimalJson).isEqualTo("\"`default_catalog`.`default_database`.`MyType`\"");
 
         // catalog lookup with miss
-        config.set(
+        tableConfig.set(
                 TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS,
                 TableConfigOptions.CatalogPlanRestore.IDENTIFIER);
         dataTypeFactoryMock.logicalType = Optional.empty();
@@ -131,7 +129,7 @@ public class LogicalTypeJsonSerdeTest {
                 .satisfies(anyCauseMatches(ValidationException.class, "No type found."));
 
         // catalog lookup
-        config.set(
+        tableConfig.set(
                 TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS,
                 TableConfigOptions.CatalogPlanRestore.IDENTIFIER);
         dataTypeFactoryMock.logicalType = Optional.of(STRUCTURED_TYPE);
@@ -139,7 +137,7 @@ public class LogicalTypeJsonSerdeTest {
                 .isEqualTo(STRUCTURED_TYPE);
 
         // maximum plan content
-        config.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, ALL);
+        tableConfig.set(TableConfigOptions.PLAN_COMPILE_CATALOG_OBJECTS, ALL);
         final String maximumJson = toJson(serdeContext, STRUCTURED_TYPE);
         final ObjectMapper mapper = new ObjectMapper();
         final JsonNode maximumJsonNode = mapper.readTree(maximumJson);
@@ -149,7 +147,7 @@ public class LogicalTypeJsonSerdeTest {
                 .isEqualTo("My original type.");
 
         // catalog lookup with miss
-        config.set(
+        tableConfig.set(
                 TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS,
                 TableConfigOptions.CatalogPlanRestore.IDENTIFIER);
         dataTypeFactoryMock.logicalType = Optional.empty();
@@ -157,7 +155,7 @@ public class LogicalTypeJsonSerdeTest {
                 .satisfies(anyCauseMatches(ValidationException.class, "No type found."));
 
         // catalog lookup
-        config.set(
+        tableConfig.set(
                 TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS,
                 TableConfigOptions.CatalogPlanRestore.IDENTIFIER);
         dataTypeFactoryMock.logicalType = Optional.of(UPDATED_STRUCTURED_TYPE);
@@ -165,7 +163,7 @@ public class LogicalTypeJsonSerdeTest {
                 .isEqualTo(UPDATED_STRUCTURED_TYPE);
 
         // no lookup
-        config.set(
+        tableConfig.set(
                 TableConfigOptions.PLAN_RESTORE_CATALOG_OBJECTS,
                 TableConfigOptions.CatalogPlanRestore.ALL);
         dataTypeFactoryMock.logicalType = Optional.of(UPDATED_STRUCTURED_TYPE);
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java
index 51aee49..b767162 100644
--- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/rules/physical/batch/PushLocalAggIntoTableSourceScanRuleTest.java
@@ -124,8 +124,7 @@ public class PushLocalAggIntoTableSourceScanRuleTest extends TableTestBase {
         // disable push down local agg
         util.getTableEnv()
                 .getConfig()
-                .getConfiguration()
-                .setBoolean(
+                .set(
                         OptimizerConfigOptions.TABLE_OPTIMIZER_SOURCE_AGGREGATE_PUSHDOWN_ENABLED,
                         false);
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala
index 56a0a88..9f4c18f 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DeadlockBreakupTest.scala
@@ -38,10 +38,10 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_SetExchangeAsBatch(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d)
@@ -52,11 +52,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_AddExchangeAsBatch_HashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -68,11 +68,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -84,12 +84,12 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_SetExchangeAsBatch_SortMergeJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,HashAgg")
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |
@@ -104,13 +104,13 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_AddExchangeAsBatch_BuildLeftSemiHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -124,8 +124,8 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_SetExchangeAsBatch_OverAgg(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r1 AS (SELECT SUM(a) OVER (PARTITION BY b ORDER BY b) AS a, b, c FROM x),
@@ -141,11 +141,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testReusedNodeIsBarrierNode(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -157,35 +157,35 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testDataStreamReuse_SetExchangeAsBatch(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.a AND t1.b > 10 AND t2.c LIKE 'Test%'"
     util.verifyExecPlan(sqlQuery)
   }
 
   @Test
   def testDataStreamReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false))
     val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.b"
     util.verifyExecPlan(sqlQuery)
   }
 
   @Test
   def testDataStreamReuse_AddExchangeAsBatch_HashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false))
     val sqlQuery = "SELECT * FROM t INTERSECT SELECT * FROM t"
     util.verifyExecPlan(sqlQuery)
   }
 
   @Test
   def testSubplanReuse_BuildAndProbeNoCommonSuccessors_HashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       s"""
@@ -202,9 +202,9 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_AddSingletonExchange(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,HashAgg")
     val sqlQuery =
       s"""
@@ -219,9 +219,9 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_DeadlockCausedByReusingExchange(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       s"""
@@ -234,11 +234,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_DeadlockCausedByReusingExchangeInAncestor(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_MULTIPLE_INPUT_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala
index 8266cda..7183c92 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/LegacySinkTest.scala
@@ -46,8 +46,9 @@ class LegacySinkTest extends TableTestBase {
   @Test
   def testMultiSinks(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
+      Boolean.box(true))
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
     util.tableEnv.registerTable("table1", table1)
     val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala
index c4f2bbb..90024e8 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/MultipleInputCreationTest.scala
@@ -44,12 +44,12 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
     util.addTableSource[(Int, Long, String, Int)]("y", 'd, 'e, 'f, 'ny)
     util.addTableSource[(Int, Long, String, Int)]("z", 'g, 'h, 'i, 'nz)
     util.addDataStream[(Int, Long, String)]("t", 'a, 'b, 'c)
-    util.tableConfig.getConfiguration.set(ExecutionOptions.BATCH_SHUFFLE_MODE, shuffleMode)
+    util.tableConfig.set(ExecutionOptions.BATCH_SHUFFLE_MODE, shuffleMode)
   }
 
   @Test
   def testBasicMultipleInput(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sql =
       """
@@ -71,9 +71,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
     //                 \-> [J -> J] -> [Agg -> J -/
     //                      |    |             |
     //                      y    t             y
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,SortAgg")
     val sql =
       """
@@ -102,7 +102,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testJoinWithAggAsProbe(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sql =
       """
@@ -119,7 +119,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
   @Test
   def testKeepMultipleInputWithOneMemberForChainableSource(): Unit = {
     createChainableTableSource()
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sql = "SELECT * FROM chainable LEFT JOIN x ON chainable.a = x.a"
     util.verifyExecPlan(sql)
@@ -127,7 +127,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testAvoidIncludingUnionFromInputSide(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sql =
       """
@@ -141,7 +141,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
   @Test
   def testIncludeUnionForChainableSource(): Unit = {
     createChainableTableSource()
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sql =
       """
@@ -154,7 +154,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testAvoidIncludingCalcAfterNonChainableSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sql =
       """
@@ -169,7 +169,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
   @Test
   def testIncludeCalcForChainableSource(): Unit = {
     createChainableTableSource()
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sql =
       """
@@ -183,7 +183,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testAvoidIncludingSingleton(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,HashAgg")
     val sql =
       """
@@ -201,7 +201,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testNoPriorityConstraint(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     val sql =
       """
@@ -214,7 +214,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testRelatedInputs(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sql =
       """
@@ -232,7 +232,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testRelatedInputsWithAgg(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin,SortAgg")
     val sql =
       """
@@ -250,9 +250,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testRemoveRedundantUnion(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sql =
       """
@@ -271,9 +271,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testRemoveOneInputOperatorFromRoot(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sql =
       """
@@ -290,7 +290,7 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testCleanUpMultipleInputWithOneMember(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sql =
       """
@@ -307,9 +307,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
   @Test
   def testKeepUsefulUnion(): Unit = {
     createChainableTableSource()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sql =
       """
@@ -326,9 +326,9 @@ class MultipleInputCreationTest(shuffleMode: BatchShuffleMode) extends TableTest
 
   @Test
   def testDeadlockCausedByExchangeInAncestor(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sql =
       """
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala
index ad1f32b..0038f52 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveCollationTest.scala
@@ -59,13 +59,13 @@ class RemoveCollationTest extends TableTestBase {
       FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
 
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true))
   }
 
   @Test
   def testRemoveCollation_OverWindowAgg(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,HashAgg")
     val sqlQuery =
       """
@@ -81,7 +81,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Aggregate(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     val sqlQuery =
       """
@@ -93,7 +93,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Aggregate_1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     val sqlQuery =
       """
@@ -105,7 +105,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Sort(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT a, b, COUNT(c) AS cnt FROM x GROUP BY a, b)
@@ -117,9 +117,9 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Aggregate_3(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x ORDER BY a, b)
@@ -130,7 +130,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -145,7 +145,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -174,7 +174,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_4(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -189,7 +189,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_Singleton(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -204,7 +204,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -220,7 +220,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -236,7 +236,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -252,7 +252,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -268,7 +268,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins3(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     util.addTableSource("tb1",
       Array[TypeInformation[_]](
@@ -344,7 +344,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Correlate1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
@@ -358,7 +358,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Correlate2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
@@ -373,7 +373,7 @@ class RemoveCollationTest extends TableTestBase {
   @Test
   def testRemoveCollation_Correlate3(): Unit = {
     // do not remove shuffle
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala
index 39951be..6d0bb65 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/RemoveShuffleTest.scala
@@ -45,13 +45,13 @@ class RemoveShuffleTest extends TableTestBase {
       Array("d", "e", "f"),
       FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false))
   }
 
   @Test
   def testRemoveHashShuffle_OverWindowAgg(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -68,7 +68,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_MultiOverWindowAgg(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -85,10 +85,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_OverWindowAgg_PartialKey(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true))
     // push down HashExchange[c] into HashAgg
     val sqlQuery =
       """
@@ -105,10 +105,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Agg_PartialKey(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true))
     // push down HashExchange[c] into HashAgg
     val sqlQuery =
       """
@@ -120,11 +120,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashAggregate(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -135,11 +135,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashAggregate_1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -150,11 +150,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashAggregate_2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -165,11 +165,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortAggregate(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -180,11 +180,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortAggregate_1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -195,11 +195,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortAggregate_2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -210,10 +210,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -224,10 +224,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin_LOJ(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -238,10 +238,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin_ROJ(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -252,7 +252,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin_FOJ(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     val sqlQuery =
       """
@@ -264,11 +264,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -279,7 +279,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_BroadcastHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -291,11 +291,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_LOJ(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x left join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -306,11 +306,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_ROJ(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x right join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -321,11 +321,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_FOJ(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x full join (SELECT * FROM y WHERE e = 2) r on a = d)
@@ -336,11 +336,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r1 AS (SELECT a, c, sum(b) FROM x group by a, c),
@@ -352,7 +352,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_NestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -364,13 +364,13 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Join_PartialKey(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT d, count(f) as cnt FROM y GROUP BY d)
@@ -387,7 +387,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Union(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -402,7 +402,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
     val sqlQuery =
       """
@@ -417,10 +417,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_PartialKey1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |SELECT a, SUM(b) FROM (
@@ -434,10 +434,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_PartialKey2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(false))
     val sqlQuery =
       """
         |SELECT * FROM (
@@ -451,10 +451,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_PartialKey3(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |SELECT * FROM (
@@ -468,7 +468,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_Singleton1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
     val sqlQuery =
       """
@@ -483,7 +483,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_Singleton2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
     val sqlQuery =
       """
@@ -498,11 +498,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Correlate1(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
       """
@@ -515,11 +515,11 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Correlate2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
       """
@@ -533,11 +533,11 @@ class RemoveShuffleTest extends TableTestBase {
   @Test
   def testRemoveHashShuffle_Correlate3(): Unit = {
     // do not remove shuffle
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
       """
@@ -555,14 +555,14 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveSingletonShuffle_HashAgg(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
     util.verifyExecPlan("SELECT MAX(b) FROM (SELECT SUM(b) AS b FROM x)")
   }
 
   @Test
   def testRemoveSingletonShuffle_SortAgg(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
     util.verifyExecPlan("SELECT MAX(b) FROM (SELECT SUM(b) AS b FROM x)")
   }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala
index 5bf4639..b5b7abe 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SetOperatorsTest.scala
@@ -34,7 +34,7 @@ class SetOperatorsTest extends TableTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
     util.addTableSource[(Int, Long, String)]("T1", 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String)]("T2", 'd, 'e, 'f)
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala
index 070de14d..8cf961d 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortLimitTest.scala
@@ -30,66 +30,66 @@ class SortLimitTest extends TableTestBase {
 
   private val util = batchTestUtil()
   util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-  util.tableEnv.getConfig.getConfiguration.setInteger(
-    ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200)
+  util.tableEnv.getConfig.set(
+    ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(200))
 
   @Test
   def testNonRangeSortWithoutOffset(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5")
   }
 
   @Test
   def testNonRangeSortWithLimit0(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0")
   }
 
   @Test
   def testNonRangeSortOnlyWithOffset(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS")
   }
 
   @Test
   def testNoneRangeSortWithOffsetLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1")
   }
 
   @Test
   def testNoneRangeSortWithOffsetLimit0(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1")
   }
 
   @Test
   def testRangeSortOnWithoutOffset(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5")
   }
 
   @Test
   def testRangeSortOnWithLimit0(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0")
   }
 
   @Test
   def testRangeSortOnlyWithOffset(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS")
   }
 
   @Test
   def testRangeSortWithOffsetLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1")
   }
 
   @Test
   def testRangeSortWithOffsetLimit0(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1")
   }
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala
index 55401f3..d2f13cc 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SortTest.scala
@@ -33,44 +33,44 @@ class SortTest extends TableTestBase {
 
   @Test
   def testNonRangeSortOnSingleFieldWithoutForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setInteger(
-      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(-1))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
 
   @Test
   def testNonRangeSortOnMultiFieldsWithoutForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      TABLE_EXEC_RANGE_SORT_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setInteger(
-      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1)
+    util.tableEnv.getConfig.set(
+      TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(-1))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC, b")
   }
 
   @Test
   def testNonRangeSortWithForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      TABLE_EXEC_RANGE_SORT_ENABLED, false)
-    util.tableEnv.getConfig.getConfiguration.setInteger(
-      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200)
+    util.tableEnv.getConfig.set(
+      TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(false))
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(200))
     util.verifyExecPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
 
   @Test
   def testRangeSortWithoutForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setInteger(
-      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, -1)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(-1))
     // exec node does not support range sort yet, so we verify rel plan here
     util.verifyRelPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
 
   @Test
   def testRangeSortWithForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setInteger(
-      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, 200)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SORT_DEFAULT_LIMIT, Integer.valueOf(200))
     // exec node does not support range sort yet, so we verify rel plan here
     util.verifyRelPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala
index 33466b3..5180190 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SubplanReuseTest.scala
@@ -37,18 +37,18 @@ class SubplanReuseTest extends TableTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f)
   }
 
   @Test
   def testDisableSubplanReuse(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false))
     val sqlQuery =
       """
         |WITH r AS (
@@ -61,8 +61,8 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseWithDifferentRowType(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     // can not reuse because of different row-type
     val sqlQuery =
       """
@@ -75,8 +75,8 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testEnableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d)
@@ -87,8 +87,8 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testDisableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     val sqlQuery =
       """
         |WITH t AS (SELECT * FROM x, y WHERE x.a = y.d)
@@ -100,9 +100,9 @@ class SubplanReuseTest extends TableTestBase {
   @Test
   def testSubplanReuseOnSourceWithLimit(): Unit = {
     // TODO re-check this plan after PushLimitIntoTableSourceScanRule is introduced
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -126,7 +126,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnCalc(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -166,7 +166,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnExchange(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -180,7 +180,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnHashAggregate(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
     val sqlQuery =
       """
@@ -192,7 +192,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnSortAggregate(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
     val sqlQuery =
       """
@@ -222,7 +222,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnSort(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_RANGE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(TABLE_EXEC_RANGE_SORT_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM x GROUP BY c ORDER BY a, b DESC)
@@ -233,7 +233,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -276,10 +276,10 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnSortMergeJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalSortMergeJoinRule.TABLE_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -290,7 +290,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -302,7 +302,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnNestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -352,7 +352,7 @@ class SubplanReuseTest extends TableTestBase {
   @Test
   def testSubplanReuseOnCorrelate(): Unit = {
     util.addFunction("str_split", new StringSplit())
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -391,7 +391,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testNestedSubplanReuse(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -427,7 +427,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testBreakupDeadlockOnHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -439,7 +439,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testBreakupDeadlockOnNestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -451,15 +451,15 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testEnableReuseTableSourceOnNewSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
     testReuseOnNewSource()
   }
 
   @Test
   def testDisableReuseTableSourceOnNewSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     testReuseOnNewSource()
   }
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala
index 8373789..7a9d275 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSinkTest.scala
@@ -72,8 +72,9 @@ class TableSinkTest extends TableTestBase {
          |)
          |""".stripMargin)
 
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
+      Boolean.box(true))
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
     util.tableEnv.createTemporaryView("table1", table1)
     val stmtSet = util.tableEnv.createStatementSet()
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala
index 0edefde..ca28de3 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/TableSourceTest.scala
@@ -220,8 +220,9 @@ class TableSourceTest extends TableTestBase {
 
   @Test
   def testTableHintWithDigestReuseForLogicalTableScan(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
+      Boolean.box(true))
     util.tableEnv.executeSql(
       s"""
          |CREATE TABLE MySink (
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala
index 7b82fee..35fef12 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/GroupWindowTest.scala
@@ -39,7 +39,7 @@ class GroupWindowTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
     util.addFunction("countFun", new CountAggFunction)
     util.addTableSource[(Int, Timestamp, Int, Long)]("MyTable", 'a, 'b, 'c, 'd)
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala
index c972b67..1239ce0 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/HashAggregateTest.scala
@@ -36,9 +36,9 @@ class HashAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe
   @Before
   def before(): Unit = {
     // disable sort agg
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
   }
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala
index e0dfe72..af072fa 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/SortAggregateTest.scala
@@ -35,9 +35,9 @@ class SortAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTe
   @Before
   def before(): Unit = {
     // disable hash agg
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
   }
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
index 07d9b85..f738ec8 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashJoinTest.scala
@@ -27,9 +27,10 @@ class BroadcastHashJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(Long.MaxValue))
+    Long.box(10)
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, ShuffleHashJoin")
   }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
index 6678332..5b7730b 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
@@ -27,9 +27,9 @@ class BroadcastHashSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(Long.MaxValue))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, ShuffleHashJoin")
     // the result plan may contains NestedLoopJoin (singleRowJoin)
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
index 3dc3e7d..6028eb4 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/LookupJoinTest.scala
@@ -329,8 +329,8 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase {
 
   @Test
   def testReusing(): Unit = {
-    testUtil.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
+    testUtil.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
     val sql1 =
       """
         |SELECT b, a, sum(c) c, sum(d) d, PROCTIME() as proctime
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala
index ff6b724..3b45288 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopJoinTest.scala
@@ -26,7 +26,7 @@ class NestedLoopJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
   }
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
index c8548f8..b884b31 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
@@ -26,7 +26,7 @@ class NestedLoopSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
   }
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
index c262b51..4b7966b 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashJoinTest.scala
@@ -27,7 +27,7 @@ class ShuffledHashJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, BroadcastHashJoin")
   }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
index f4814c4..8315cfe 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
@@ -27,7 +27,7 @@ class ShuffledHashSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, BroadcastHashJoin")
     // the result plan may contains NestedLoopJoin (singleRowJoin)
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala
index b8b0a70..2eacfc9 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeJoinTest.scala
@@ -27,7 +27,7 @@ class SortMergeJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
   }
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
index e358c7b..4a79b98 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
@@ -27,7 +27,7 @@ class SortMergeSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
     // the result plan may contains NestedLoopJoin (singleRowJoin)
     // which is converted by BatchExecSingleRowJoinRule
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala
index 0cf6659..2137811 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/JoinReorderTestBase.scala
@@ -70,8 +70,8 @@ abstract class JoinReorderTestBase extends TableTestBase {
         "b5" -> new ColumnStats(200L, 0L, 8.0, 8, null, null)
       ))).build())
 
-    util.getTableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, true)
+    util.getTableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_JOIN_REORDER_ENABLED, Boolean.box(true))
   }
 
   @Test
@@ -266,8 +266,8 @@ abstract class JoinReorderTestBase extends TableTestBase {
         "b8" -> builderB.build()
       ))).build())
 
-    util.getTableEnv.getConfig.getConfiguration.setLong(
-      JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, 10000L)
+    util.getTableEnv.getConfig.set(
+      JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, Long.box(10000))
     val sql =
       s"""
          |SELECT * FROM T6
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala
index c33a9d2..50fc563 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala
@@ -47,7 +47,7 @@ class TableFactoryTest(isBatch: Boolean) extends TableTestBase {
       ObjectIdentifier.of("cat", "default", "t1"),
       ObjectIdentifier.of("cat", "default", "t2"),
       isBatch)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TestContextTableFactory.REQUIRED_KEY, true)
+    util.tableEnv.getConfig.set(TestContextTableFactory.REQUIRED_KEY, Boolean.box(true))
     util.tableEnv.registerCatalog("cat", new GenericInMemoryCatalog("default") {
       override def getTableFactory: Optional[TableFactory] = Optional.of(factory)
     })
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala
index 0ff55d1..ed7e224 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala
@@ -75,9 +75,9 @@ class OptionsHintTest(param: Param)
 
   @Test
   def testOptionsWithGlobalConfDisabled(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
+    util.tableEnv.getConfig.set(
       TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED,
-      false)
+      Boolean.box(false))
     expectedException.expect(isA(classOf[ValidationException]))
     expectedException.expectMessage(s"OPTIONS hint is allowed only when "
       + s"${TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED.key} is set to true")
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
index b459371..8513973 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
@@ -52,8 +52,8 @@ class JoinDeriveNullFilterRuleTest extends TableTestBase {
         .build()
     )
 
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, 2000000)
+    util.tableEnv.getConfig.set(
+      JoinDeriveNullFilterRule.TABLE_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, Long.box(2000000))
     util.addTableSource("MyTable1",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING, Types.INT, Types.LONG),
       Array("a1", "b1", "c1", "d1", "e1"),
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala
index d809dc4..26d4324 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/SplitAggregateRuleTest.scala
@@ -33,8 +33,8 @@ class SplitAggregateRuleTest extends TableTestBase {
   private val util = streamTestUtil()
   util.addTableSource[(Long, Int, String)]("MyTable", 'a, 'b, 'c)
   util.buildStreamProgram(FlinkStreamProgram.PHYSICAL)
-  util.tableEnv.getConfig.getConfiguration.setBoolean(
-    OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+  util.tableEnv.getConfig.set(
+    OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
 
   @Test
   def testSingleDistinctAgg(): Unit = {
@@ -163,16 +163,16 @@ class SplitAggregateRuleTest extends TableTestBase {
 
   @Test
   def testBucketsConfiguration(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setInteger(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM, 100)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM, Integer.valueOf(100))
     val sqlQuery = "SELECT COUNT(DISTINCT c) FROM MyTable"
     util.verifyRelPlan(sqlQuery)
   }
 
   @Test
   def testMultipleDistinctAggOnSameColumn(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sqlQuery =
       s"""
          |SELECT
@@ -189,8 +189,8 @@ class SplitAggregateRuleTest extends TableTestBase {
 
   @Test
   def testAggFilterClauseBothWithAvgAndCount(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sqlQuery =
       s"""
          |SELECT
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala
index c0c1128..9d93773 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala
@@ -51,9 +51,9 @@ class EnforceLocalHashAggRuleTest extends EnforceLocalAggRuleTestBase {
       .replaceBatchProgram(program).build()
     util.tableEnv.getConfig.setPlannerConfig(calciteConfig)
     // only enabled HashAgg
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE")
   }
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala
index e0eeca2..29520c5 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala
@@ -55,9 +55,9 @@ class EnforceLocalSortAggRuleTest extends EnforceLocalAggRuleTestBase {
       .replaceBatchProgram(program).build()
     util.tableEnv.getConfig.setPlannerConfig(calciteConfig)
     // only enabled SortAgg
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashAgg")
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE")
   }
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
index 1060fa8..fba9798 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
@@ -40,11 +40,11 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase {
 
   @Test
   def testRemoveRedundantLocalHashAgg_ShuffleKeyFromJoin(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.box(-1))
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x, y WHERE a = d AND c LIKE 'He%')
@@ -55,10 +55,10 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase {
 
   @Test
   def testRemoveRedundantLocalHashAgg_ShuffleKeyFromRank(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |SELECT a, SUM(b) FROM (
@@ -72,10 +72,10 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase {
 
   @Test
   def testUsingLocalAggCallFilters(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED, Boolean.box(true))
     val sqlQuery = "SELECT d, MAX(e), MAX(e) FILTER (WHERE a < 10), COUNT(DISTINCT c),\n" +
       "COUNT(DISTINCT c) FILTER (WHERE a > 5), COUNT(DISTINCT b) FILTER (WHERE b > 3)\n" +
       "FROM z GROUP BY d"
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala
index 15f4293..a7fbbee 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.planner.plan.rules.physical.stream
 
 import org.apache.flink.api.common.time.Time
-import org.apache.flink.table.api.{ExplainDetail, _}
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder
 import org.apache.flink.table.planner.plan.optimize.program.FlinkChangelogModeInferenceProgram
@@ -162,7 +162,7 @@ class ChangelogModeInferenceTest extends TableTestBase {
   def testTwoLevelGroupByLocalGlobalOn(): Unit = {
       util.enableMiniBatch()
       util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
-      util.tableEnv.getConfig.getConfiguration.setString(
+      util.tableEnv.getConfig.set(
         OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
         AggregatePhaseStrategy.TWO_PHASE.toString)
     // two level unbounded groupBy
@@ -223,9 +223,9 @@ class ChangelogModeInferenceTest extends TableTestBase {
 
   @Test
   def testPropagateUpdateKindAmongRelNodeBlocks(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
+    util.tableEnv.getConfig.set(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
-      true)
+      Boolean.box(true))
     util.addTable(
       """
         |create table sink1 (
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala
index a3ec3d7..e1cdf31 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ExpandWindowTableFunctionTransposeRuleTest.scala
@@ -83,8 +83,8 @@ class ExpandWindowTableFunctionTransposeRuleTest extends TableTestBase {
          |)
          |""".stripMargin)
 
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
   }
 
   @Test
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
index 414b044..1a39444 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
@@ -115,8 +115,8 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testSingleSinkSplitOnUnion(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
 
     val sqlQuery = "SELECT SUM(a) AS total_sum FROM " +
       "(SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
@@ -128,8 +128,9 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinks1(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
+      Boolean.box(true))
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
     util.tableEnv.registerTable("table1", table1)
     val table2 = util.tableEnv.sqlQuery("SELECT SUM(sum_a) AS total_sum FROM table1")
@@ -151,8 +152,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinks2(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(true))
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b as b1 FROM MyTable WHERE a <= 10")
@@ -177,8 +178,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinks3(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(true))
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a AS a1, b AS b1 FROM MyTable WHERE a <= 10")
@@ -231,8 +232,9 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinks5(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
+      Boolean.box(true))
     // test with non-deterministic udf
     util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf())
     val table1 = util.tableEnv.sqlQuery("SELECT random_udf(a) AS a, c FROM MyTable")
@@ -256,8 +258,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinksWithUDTF(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
     util.addFunction("split", new TableFunc1)
     val sqlQuery1 =
       """
@@ -300,8 +302,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinksSplitOnUnion1(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
 
     val table = util.tableEnv.sqlQuery(
       "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1")
@@ -325,10 +327,11 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinksSplitOnUnion2(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
+      Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
     val sqlQuery1 =
@@ -367,8 +370,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinksSplitOnUnion3(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
     val sqlQuery1 = "SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1"
@@ -402,8 +405,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiSinksSplitOnUnion4(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
     val sqlQuery =
@@ -509,8 +512,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testMultiLevelViews(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
     util.tableEnv.registerTable("TempTable1", table1)
@@ -552,8 +555,8 @@ class DagOptimizationTest extends TableTestBase {
   @Test
   def testSharedUnionNode(): Unit = {
     val stmtSet = util.tableEnv.createStatementSet()
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_ENABLED, Boolean.box(false))
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
     util.tableEnv.registerTable("TempTable1", table1)
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala
index a6ec4e8..027dfac 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeduplicateTest.scala
@@ -70,7 +70,7 @@ class DeduplicateTest extends TableTestBase {
 
   @Test
   def testLastRowWithWindowOnRowtime(): Unit = {
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig
       .set(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(500))
     util.addTable(
       """
@@ -125,9 +125,9 @@ class DeduplicateTest extends TableTestBase {
 
   @Test
   def testMiniBatchInferFirstRowOnRowtime(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setLong(TABLE_EXEC_MINIBATCH_SIZE, 3L)
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_SIZE, Long.box(3L))
+    util.tableEnv.getConfig.set(
       TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     val ddl =
       s"""
@@ -175,9 +175,9 @@ class DeduplicateTest extends TableTestBase {
 
   @Test
   def testMiniBatchInferLastRowOnRowtime(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setLong(TABLE_EXEC_MINIBATCH_SIZE, 3L)
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(TABLE_EXEC_MINIBATCH_SIZE, Long.box(3L))
+    util.tableEnv.getConfig.set(
       TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     val ddl =
       s"""
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
index 0cf621c..988ac69 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
@@ -72,13 +72,13 @@ class MiniBatchIntervalInferTest extends TableTestBase {
          |""".stripMargin)
 
     // enable mini-batch
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
   }
 
   @Test
   def testMiniBatchOnly(): Unit = {
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig
       .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable1 GROUP BY b"
     util.verifyExecPlan(sql)
@@ -86,7 +86,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testMiniBatchOnlyForDataStream(): Unit = {
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig
         .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyDataStream1 GROUP BY b"
     util.verifyExecPlan(sql)
@@ -94,7 +94,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testRedundantWatermarkDefinition(): Unit = {
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig
         .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM wmTable1 GROUP BY b"
     util.verifyExecPlan(sql)
@@ -103,7 +103,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   @Test
   def testWindowWithEarlyFire(): Unit = {
     val tableConfig = util.tableEnv.getConfig
-    tableConfig.getConfiguration
+    tableConfig
         .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     withEarlyFireDelay(tableConfig, Time.milliseconds(500))
     val sql =
@@ -124,7 +124,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testWindowCascade(): Unit = {
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig
         .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(3))
     val sql =
       """
@@ -144,7 +144,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testIntervalJoinWithMiniBatch(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
 
     val sql =
@@ -165,7 +165,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testRowtimeRowsOverWithMiniBatch(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
 
     val sql =
@@ -187,7 +187,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
     util.addTableWithWatermark("Orders", util.tableEnv.from("MyDataStream1"), "rowtime", 0)
     util.addTableWithWatermark("RatesHistory", util.tableEnv.from("MyDataStream2"), "rowtime", 0)
 
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
 
     util.addTemporarySystemFunction(
@@ -212,7 +212,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   @Test
   def testMultiOperatorNeedsWatermark1(): Unit = {
     // infer result: miniBatchInterval=[Rowtime, 0ms]
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
 
     val sql =
@@ -236,7 +236,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testMultiOperatorNeedsWatermark2(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(6))
 
     val sql =
@@ -271,7 +271,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testMultiOperatorNeedsWatermark3(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(6))
 
     val sql =
@@ -326,10 +326,9 @@ class MiniBatchIntervalInferTest extends TableTestBase {
          |)
          |""".stripMargin)
 
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(500))
-    util.tableEnv.getConfig.getConfiguration.setLong(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, 300L)
+    util.tableEnv.getConfig.set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_SIZE, Long.box(300L))
 
     val table1 = util.tableEnv.sqlQuery(
       """
@@ -393,7 +392,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   @Test
   def testMiniBatchOnDataStreamWithRowTime(): Unit = {
     util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'str, 'rowtime.rowtime)
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig
       .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     val sql =
       """
@@ -409,7 +408,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   @Test
   def testOverWindowMiniBatchOnDataStreamWithRowTime(): Unit = {
     util.addDataStream[(Long, Int, String)]("T1", 'long, 'int, 'str, 'rowtime.rowtime)
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig
       .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(3))
     val sql =
       """
@@ -428,7 +427,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   private def withEarlyFireDelay(tableConfig: TableConfig, interval: Time): Unit = {
     val intervalInMillis = interval.toMilliseconds
-    val earlyFireDelay: Duration = tableConfig.getConfiguration
+    val earlyFireDelay: Duration = tableConfig
       .getOptional(TABLE_EXEC_EMIT_EARLY_FIRE_DELAY)
       .orElse(null)
     if (earlyFireDelay != null && (earlyFireDelay.toMillis != intervalInMillis)) { //
@@ -436,8 +435,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
       throw new RuntimeException("Currently not support different earlyFireInterval configs in " +
         "one job")
     }
-    tableConfig.getConfiguration.setBoolean(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED, Boolean.box(true))
-    tableConfig.getConfiguration.set(
-      TABLE_EXEC_EMIT_EARLY_FIRE_DELAY, Duration.ofMillis(intervalInMillis))
+    tableConfig.set(TABLE_EXEC_EMIT_EARLY_FIRE_ENABLED, Boolean.box(true))
+    tableConfig.set(TABLE_EXEC_EMIT_EARLY_FIRE_DELAY, Duration.ofMillis(intervalInMillis))
   }
 }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
index 4cb7b66..c4b14ac 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
@@ -67,9 +67,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
 
   @Test
   def testMaxWithRetractOptimizeWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig
       .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(100))
     val query = "SELECT a1, max(a3) from (SELECT a1, a2, max(a3) as a3 FROM A GROUP BY a1, a2) " +
       "group by a1"
@@ -78,9 +78,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
 
   @Test
   def testMinWithRetractOptimizeWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig
       .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(100))
     val query = "SELECT min(a3) from (SELECT a1, a2, min(a3) as a3 FROM A GROUP BY a1, a2)"
     util.verifyRelPlan(query, ExplainDetail.CHANGELOG_MODE)
@@ -88,9 +88,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
 
   @Test
   def testMinCanNotOptimizeWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig
       .set(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofMillis(100))
     val query =
       "SELECT a1, MIN(a3) FROM (SELECT a1, a2, MAX(a3) AS a3 FROM A GROUP BY a1, a2) t GROUP BY a1"
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala
index 3963652..18d292f 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala
@@ -840,8 +840,9 @@ class RankTest extends TableTestBase {
 
   @Test
   def testUpdatableRankAfterIntermediateScan(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED,
+      Boolean.box(true))
     util.tableEnv.executeSql(
       """
         |CREATE VIEW v1 AS
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala
index 12a851e..c6b0922 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SubplanReuseTest.scala
@@ -34,18 +34,18 @@ class SubplanReuseTest extends TableTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f)
   }
 
   @Test
   def testDisableSubplanReuse(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, Boolean.box(false))
     val sqlQuery =
       """
         |WITH r AS (
@@ -58,8 +58,8 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseWithDifferentRowType(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     // can not reuse because of different row-type
     val sqlQuery =
       """
@@ -72,8 +72,8 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testEnableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
     val sqlQuery =
       """
         |WITH t AS (SELECT x.a AS a, x.b AS b, y.d AS d, y.e AS e FROM x, y WHERE x.a = y.d)
@@ -84,8 +84,8 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testDisableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     val sqlQuery =
       """
         |WITH t AS (SELECT * FROM x, y WHERE x.a = y.d)
@@ -184,7 +184,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnLimit(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -303,15 +303,15 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testEnableReuseTableSourceOnNewSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(true))
     testReuseOnNewSource()
   }
 
   @Test
   def testDisableReuseTableSourceOnNewSource(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SOURCE_ENABLED, Boolean.box(false))
     testReuseOnNewSource()
   }
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala
index 598a2dd..63051cb 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala
@@ -288,8 +288,8 @@ class TableScanTest extends TableTestBase {
 
   @Test
   def testJoinOnChangelogSourceWithEventsDuplicate(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, true)
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, Boolean.box(true))
     verifyJoinOnSource("I,UB,UA")
   }
 
@@ -356,8 +356,8 @@ class TableScanTest extends TableTestBase {
 
   @Test
   def testChangelogSourceWithEventsDuplicate(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, true)
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, Boolean.box(true))
     util.addTable(
       """
         |CREATE TABLE src (
@@ -668,8 +668,8 @@ class TableScanTest extends TableTestBase {
         |  'changelog-mode' = 'I,UB,UA,D'
         |)
       """.stripMargin)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, true)
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE, Boolean.box(true))
 
     thrown.expect(classOf[TableException])
     thrown.expectMessage("Configuration 'table.exec.source.cdc-events-duplicate' is enabled " +
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala
index d5ef3f3..f760fb5 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableSinkTest.scala
@@ -504,7 +504,7 @@ class TableSinkTest extends TableTestBase {
 
   @Test def testAppendStreamToSinkWithPkNoKeyBy(): Unit = {
     val tEnv = util.tableEnv
-    tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
+    tEnv.getConfig.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
       ExecutionConfigOptions.SinkKeyedShuffle.NONE)
     tEnv.executeSql(
       """
@@ -535,7 +535,8 @@ class TableSinkTest extends TableTestBase {
   @Test def testAppendStreamToSinkWithPkForceKeyBy(): Unit = {
     util.getStreamEnv.setParallelism(4)
     val tEnv = util.tableEnv
-    tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
+    tEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
       ExecutionConfigOptions.SinkKeyedShuffle.FORCE)
     tEnv.executeSql(
       """
@@ -566,7 +567,8 @@ class TableSinkTest extends TableTestBase {
   @Test def testSingleParallelismAppendStreamToSinkWithPkForceKeyBy(): Unit = {
     util.getStreamEnv.setParallelism(1)
     val tEnv = util.tableEnv
-    tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
+    tEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
       ExecutionConfigOptions.SinkKeyedShuffle.FORCE)
     tEnv.executeSql(
       """
@@ -597,7 +599,8 @@ class TableSinkTest extends TableTestBase {
   @Test def testAppendStreamToSinkWithoutPkForceKeyBy(): Unit = {
     util.getStreamEnv.setParallelism(4)
     val tEnv = util.tableEnv
-    tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
+    tEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
       ExecutionConfigOptions.SinkKeyedShuffle.FORCE)
     tEnv.executeSql(
       """
@@ -627,7 +630,8 @@ class TableSinkTest extends TableTestBase {
   @Test def testAppendStreamToSinkWithoutPkForceKeyBySingleParallelism(): Unit = {
     util.getStreamEnv.setParallelism(4)
     val tEnv = util.tableEnv
-    tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
+    tEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
       ExecutionConfigOptions.SinkKeyedShuffle.FORCE)
     tEnv.executeSql(
       """
@@ -656,7 +660,8 @@ class TableSinkTest extends TableTestBase {
   @Test def testChangelogStreamToSinkWithPkDifferentParallelism(): Unit = {
     util.getStreamEnv.setParallelism(1)
     val tEnv = util.tableEnv
-    tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
+    tEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
       ExecutionConfigOptions.SinkKeyedShuffle.AUTO)
     tEnv.executeSql(
       """
@@ -685,10 +690,12 @@ class TableSinkTest extends TableTestBase {
     util.verifyExplain(stmtSet, ExplainDetail.JSON_EXECUTION_PLAN)
   }
 
-  @Test def testChangelogStreamToSinkWithPkSingleParallelism(): Unit = {
+  @Test
+  def testChangelogStreamToSinkWithPkSingleParallelism(): Unit = {
     util.getStreamEnv.setParallelism(4)
     val tEnv = util.tableEnv
-    tEnv.getConfig.getConfiguration.set(ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
+    tEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_SINK_KEYED_SHUFFLE,
       ExecutionConfigOptions.SinkKeyedShuffle.FORCE)
     tEnv.executeSql(
       """
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
index 883415d..48274d4 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
@@ -64,18 +64,18 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testAggWithMiniBatch(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     util.verifyExecPlan("SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c)  FROM MyTable GROUP BY b")
   }
 
   @Test
   def testAggAfterUnionWithMiniBatch(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
     val query =
       """
@@ -97,9 +97,9 @@ class AggregateTest extends TableTestBase {
   @Test
   def testLocalGlobalAggAfterUnion(): Unit = {
     // enable local global optimize
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
 
     val sql =
@@ -130,9 +130,9 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testAggWithFilterClauseWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConfiguration.set(
+    util.tableEnv.getConfig.set(
+      ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+    util.tableEnv.getConfig.set(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
 
     val sql =
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
index bc661d1..795d268 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
@@ -44,13 +44,14 @@ class DistinctAggregateTest(
   def before(): Unit = {
     util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
     util.enableMiniBatch()
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseEnforcer.toString)
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, splitDistinctAggEnabled)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED,
+      Boolean.box(splitDistinctAggEnabled))
     // disable incremental agg
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, false)
+    util.tableEnv.getConfig.set(
+      IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, Boolean.box(false))
   }
 
   @Test
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala
index 90739d5..d0a039f 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/GroupWindowTest.scala
@@ -433,8 +433,8 @@ class GroupWindowTest extends TableTestBase {
 
   @Test
   def testWindowAggregateWithLateFire(): Unit = {
-    util.conf.getConfiguration.setBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, true)
-    util.conf.getConfiguration.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5))
+    util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, Boolean.box(true))
+    util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5))
     util.conf.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
     val sql =
       """
@@ -447,9 +447,9 @@ class GroupWindowTest extends TableTestBase {
 
   @Test
   def testWindowAggregateWithAllowLateness(): Unit = {
-    util.conf.getConfiguration.setBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, true)
-    util.conf.getConfiguration.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5))
-    util.conf.getConfiguration.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofHours(1))
+    util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, Boolean.box(true))
+    util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5))
+    util.conf.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofHours(1))
     val sql =
       """
         |SELECT TUMBLE_START(`rowtime`, INTERVAL '1' SECOND), COUNT(*) cnt
@@ -461,9 +461,9 @@ class GroupWindowTest extends TableTestBase {
 
   @Test
   def testWindowAggregateWithInvalidAllowLateness(): Unit = {
-    util.conf.getConfiguration.setBoolean(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, true)
-    util.conf.getConfiguration.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5))
-    util.conf.getConfiguration.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofSeconds(1))
+    util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_ENABLED, Boolean.box(true))
+    util.conf.set(TABLE_EXEC_EMIT_LATE_FIRE_DELAY, Duration.ofSeconds(5))
+    util.conf.set(TABLE_EXEC_EMIT_ALLOW_LATENESS, Duration.ofSeconds(1))
     val sql =
       """
         |SELECT TUMBLE_START(`rowtime`, INTERVAL '1' SECOND), COUNT(*) cnt
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala
index 75ebee1..8863204 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/IncrementalAggregateTest.scala
@@ -37,8 +37,8 @@ class IncrementalAggregateTest(
   override def before(): Unit = {
     super.before()
     // enable incremental agg
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      IncrementalAggregateRule.TABLE_OPTIMIZER_INCREMENTAL_AGG_ENABLED, Boolean.box(true))
   }
 }
 
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala
index 93c157d..9675761 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/TwoStageAggregateTest.scala
@@ -35,7 +35,7 @@ class TwoStageAggregateTest extends TableTestBase {
   def before(): Unit = {
     util.enableMiniBatch()
     util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
       AggregatePhaseStrategy.TWO_PHASE.toString)
   }
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala
index d1b9a62..5eddd71 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala
@@ -69,7 +69,7 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
          |""".stripMargin)
 
     // set agg-phase strategy
-    util.tableEnv.getConfig.getConfiguration.setString(
+    util.tableEnv.getConfig.set(
       OptimizerConfigOptions.TABLE_OPTIMIZER_AGG_PHASE_STRATEGY,
       aggPhaseEnforcer.toString)
   }
@@ -284,8 +284,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testTumble_DistinctSplitEnabled(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sql =
       """
         |SELECT
@@ -304,8 +304,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testTumble_DistinctOnWindowColumns(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     // window_time is used in agg arg, thus we shouldn't merge WindowTVF into WindowAggregate.
     // actually, after expanded, there's HASH_CODE(window_time),
     // and thus we shouldn't transpose WindowTVF and Expand too.
@@ -328,8 +328,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
   def testTumble_DoNotSplitProcessingTimeWindow(): Unit = {
     assumeTrue(isTwoPhase)
     // the processing-time window aggregate with distinct shouldn't be split into two-level agg
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sql =
       """
         |SELECT
@@ -426,8 +426,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testCumulate_DistinctSplitEnabled(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sql =
       """
         |SELECT
@@ -488,8 +488,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testHop_DistinctSplitEnabled(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sql =
       """
         |SELECT
@@ -792,8 +792,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testTumble_GroupingSetsDistinctSplitEnabled(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sql =
       """
         |SELECT
@@ -855,8 +855,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testCantMergeWindowTVF_GroupingSetsDistinctOnWindowColumns(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     // window_time is used in agg arg, thus we shouldn't merge WindowTVF into WindowAggregate.
     // actually, after expanded, there's HASH_CODE(window_time),
     // and thus we shouldn't transpose WindowTVF and Expand too.
@@ -891,8 +891,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testHop_GroupingSets_DistinctSplitEnabled(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sql =
       """
         |SELECT
@@ -954,8 +954,8 @@ class WindowAggregateTest(aggPhaseEnforcer: AggregatePhaseStrategy) extends Tabl
 
   @Test
   def testCumulate_GroupingSets_DistinctSplitEnabled(): Unit = {
-    util.tableEnv.getConfig.getConfiguration.setBoolean(
-      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
+    util.tableEnv.getConfig.set(
+      OptimizerConfigOptions.TABLE_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, Boolean.box(true))
     val sql =
       """
         |SELECT
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala
index 9324b92..583c4cb 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingWithMiniBatchTestBase.scala
@@ -39,9 +39,9 @@ abstract class StreamingWithMiniBatchTestBase(
     val tableConfig = tEnv.getConfig
     miniBatch match {
       case MiniBatchOn =>
-        tableConfig.getConfiguration.setBoolean(TABLE_EXEC_MINIBATCH_ENABLED, true)
-        tableConfig.getConfiguration.set(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
-        tableConfig.getConfiguration.setLong(TABLE_EXEC_MINIBATCH_SIZE, 3L)
+        tableConfig.set(TABLE_EXEC_MINIBATCH_ENABLED, Boolean.box(true))
+        tableConfig.set(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, Duration.ofSeconds(1))
+        tableConfig.set(TABLE_EXEC_MINIBATCH_SIZE, Long.box(3))
       case MiniBatchOff =>
         tableConfig.getConfiguration.removeConfig(TABLE_EXEC_MINIBATCH_ALLOW_LATENCY)
     }

[flink] 05/08: [FLINK-26777][table-planner] Remove `PlannerConfig` from `PlannerBase`

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit b7fb46469a05e8b171b2034e765365456630d92c
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 14:52:28 2022 +0200

    [FLINK-26777][table-planner] Remove `PlannerConfig` from `PlannerBase`
    
    Remove `PlannerConfig` delegation class, since `TableConfig` is now the
    one that holds the complete view of `rootConfiguration` (environment
    config) + `configuration` (the application specific configuration).
    
    This closes #19197.
    
    (cherry picked from commit f1bedbe4110c288f561f02e1909b67b694d5278b)
---
 .../table/planner/delegation/PlannerConfig.java    | 56 ----------------------
 .../planner/plan/nodes/exec/ExecNodeBase.java      |  4 +-
 .../planner/plan/nodes/exec/ExecNodeConfig.java    | 47 +++++++-----------
 .../exec/processor/DeadlockBreakupProcessor.java   |  2 +-
 .../MultipleInputNodeCreationProcessor.java        |  2 +-
 .../table/planner/delegation/PlannerBase.scala     | 13 +----
 .../table/planner/delegation/StreamPlanner.scala   |  2 +-
 .../BatchCommonSubGraphBasedOptimizer.scala        |  2 +-
 .../StreamCommonSubGraphBasedOptimizer.scala       | 10 ++--
 .../plan/metadata/FlinkRelMdHandlerTestBase.scala  |  6 +--
 .../batch/EnforceLocalHashAggRuleTest.scala        |  2 +-
 .../batch/EnforceLocalSortAggRuleTest.scala        |  2 +-
 .../sql/agg/AggregateJoinTransposeITCase.scala     |  2 +-
 .../flink/table/planner/utils/TableTestBase.scala  | 11 ++---
 14 files changed, 38 insertions(+), 123 deletions(-)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerConfig.java
deleted file mode 100644
index 3563d76..0000000
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/delegation/PlannerConfig.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.table.planner.delegation;
-
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ReadableConfig;
-
-import java.util.Optional;
-
-/**
- * Configuration view that combines the API specific table configuration and the executor
- * configuration. The table configuration has precedence.
- */
-@Internal
-public final class PlannerConfig implements ReadableConfig {
-
-    private final ReadableConfig tableConfig;
-
-    private final ReadableConfig executorConfig;
-
-    PlannerConfig(ReadableConfig tableConfig, ReadableConfig executorConfig) {
-        this.tableConfig = tableConfig;
-        this.executorConfig = executorConfig;
-    }
-
-    @Override
-    public <T> T get(ConfigOption<T> option) {
-        return tableConfig.getOptional(option).orElseGet(() -> executorConfig.get(option));
-    }
-
-    @Override
-    public <T> Optional<T> getOptional(ConfigOption<T> option) {
-        final Optional<T> tableValue = tableConfig.getOptional(option);
-        if (tableValue.isPresent()) {
-            return tableValue;
-        }
-        return executorConfig.getOptional(option);
-    }
-}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
index 337230c..dbb8810 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
@@ -129,9 +129,7 @@ public abstract class ExecNodeBase<T> implements ExecNode<T> {
                     translateToPlanInternal(
                             (PlannerBase) planner,
                             new ExecNodeConfig(
-                                    ((PlannerBase) planner).getConfiguration(),
-                                    ((PlannerBase) planner).getTableConfig(),
-                                    new Configuration()));
+                                    ((PlannerBase) planner).getTableConfig(), new Configuration()));
             if (this instanceof SingleTransformationTranslator) {
                 if (inputsContainSingleton()) {
                     transformation.setParallelism(1);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
index 183ce14..b19ec9f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
@@ -20,57 +20,44 @@ package org.apache.flink.table.planner.plan.nodes.exec;
 
 import org.apache.flink.annotation.Internal;
 import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.Configuration;
 import org.apache.flink.configuration.ReadableConfig;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.table.api.TableConfig;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
-import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
 import org.apache.flink.table.planner.delegation.PlannerBase;
-import org.apache.flink.table.planner.delegation.PlannerConfig;
+import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
 
 import java.time.ZoneId;
 import java.util.Optional;
 
 /**
- * Configuration view which is used combine the {@link PlannerConfig} with the {@link
+ * Configuration view which is used combine the {@link PlannerBase#getTableConfig()} with the {@link
  * ExecNodeBase#getNodeConfig()} configuration. The persisted configuration of the {@link ExecNode}
- * which is deserialized from the JSON plan has precedence over the {@link PlannerConfig}.
+ * which is deserialized from the JSON plan has precedence over the {@link
+ * PlannerBase#getTableConfig()}.
  */
 @Internal
 public final class ExecNodeConfig implements ReadableConfig {
 
-    private final ReadableConfig plannerConfig;
-
     // See https://issues.apache.org/jira/browse/FLINK-26190
     // Used only for the deprecated getMaxIdleStateRetentionTime to also satisfy tests which
     // manipulate maxIdleStateRetentionTime, like OverAggregateHarnessTest.
-    private final TableConfig originalTableConfig;
-    // See https://issues.apache.org/jira/browse/FLINK-26190
     private final TableConfig tableConfig;
 
     private final ReadableConfig nodeConfig;
 
-    ExecNodeConfig(
-            ReadableConfig plannerConfig, TableConfig tableConfig, ReadableConfig nodeConfig) {
-        this.plannerConfig = plannerConfig;
+    ExecNodeConfig(TableConfig tableConfig, ReadableConfig nodeConfig) {
         this.nodeConfig = nodeConfig;
-        this.originalTableConfig = tableConfig;
-        this.tableConfig = TableConfig.getDefault();
-        this.tableConfig.setNullCheck(tableConfig.getNullCheck());
-        this.tableConfig.setDecimalContext(tableConfig.getDecimalContext());
-        this.tableConfig.addConfiguration(tableConfig.getConfiguration());
-        this.tableConfig.addConfiguration((Configuration) nodeConfig);
+        this.tableConfig = tableConfig;
     }
 
     /**
-     * Return the merged {@link TableConfig} from {@link PlannerBase#getTableConfig()} and {@link
-     * ExecNodeBase#getNodeConfig()}.
+     * Return the {@link PlannerBase#getTableConfig()}.
      *
-     * @return the {@link TableConfig}.
-     * @deprecated This method is used only for {@link CodeGeneratorContext} and related methods,
-     *     which end up passing the {@link TableConfig} to the {@link CodeGeneratorContext}. It
-     *     should be removed once {@link CodeGeneratorContext#nullCheck()} is removed, since for all
-     *     other usages it's possible to use the {@link ReadableConfig}.
+     * @return the {@link PlannerBase#getTableConfig()}.
+     * @deprecated This method is used only for {@link
+     *     CommonPythonUtil#getMergedConfig(StreamExecutionEnvironment, TableConfig)}. It should be
+     *     removed when this method is refactored to accept a {@link ReadableConfig} instead.
      */
     // See https://issues.apache.org/jira/browse/FLINK-26190
     @Deprecated
@@ -80,7 +67,7 @@ public final class ExecNodeConfig implements ReadableConfig {
 
     @Override
     public <T> T get(ConfigOption<T> option) {
-        return nodeConfig.getOptional(option).orElseGet(() -> plannerConfig.get(option));
+        return nodeConfig.getOptional(option).orElseGet(() -> tableConfig.get(option));
     }
 
     @Override
@@ -89,7 +76,7 @@ public final class ExecNodeConfig implements ReadableConfig {
         if (tableValue.isPresent()) {
             return tableValue;
         }
-        return plannerConfig.getOptional(option);
+        return tableConfig.getOptional(option);
     }
 
     /** @return The duration until state which was not updated will be retained. */
@@ -99,13 +86,13 @@ public final class ExecNodeConfig implements ReadableConfig {
 
     // See https://issues.apache.org/jira/browse/FLINK-26190
     /**
-     * Using {@link #originalTableConfig} to satisify tests like {@code OverAggregateHarnessTest},
-     * which use {@code HarnessTestBase#TestTableConfig} to individually manipulate the
+     * Using {@link #tableConfig} to satisify tests like {@code OverAggregateHarnessTest}, which use
+     * {@code HarnessTestBase#TestTableConfig} to individually manipulate the
      * maxIdleStateRetentionTime. See {@link TableConfig#getMaxIdleStateRetentionTime()}.
      */
     @Deprecated
     public long getMaxIdleStateRetentionTime() {
-        return originalTableConfig.getMaxIdleStateRetentionTime();
+        return tableConfig.getMaxIdleStateRetentionTime();
     }
 
     // See https://issues.apache.org/jira/browse/FLINK-26190
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java
index 9959262..6c1e769 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/DeadlockBreakupProcessor.java
@@ -44,7 +44,7 @@ public class DeadlockBreakupProcessor implements ExecNodeGraphProcessor {
                         execGraph.getRootNodes(),
                         InputProperty.DamBehavior.END_INPUT,
                         StreamExchangeMode.BATCH,
-                        context.getPlanner().getConfiguration());
+                        context.getPlanner().getTableConfig());
         resolver.detectAndResolve();
         return execGraph;
     }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java
index 8cc3cc4..fc245d4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java
@@ -79,7 +79,7 @@ public class MultipleInputNodeCreationProcessor implements ExecNodeGraphProcesso
                             execGraph.getRootNodes(),
                             InputProperty.DamBehavior.BLOCKING,
                             StreamExchangeMode.PIPELINED,
-                            context.getPlanner().getConfiguration());
+                            context.getPlanner().getTableConfig());
             resolver.detectAndResolve();
         }
 
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
index 05d7baf..a3a5c24 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/PlannerBase.scala
@@ -103,10 +103,6 @@ abstract class PlannerBase(
   private var parser: Parser = _
   private var currentDialect: SqlDialect = getTableConfig.getSqlDialect
 
-  private val plannerConfiguration: ReadableConfig = new PlannerConfig(
-    tableConfig.getConfiguration,
-    executor.getConfiguration)
-
   @VisibleForTesting
   private[flink] val plannerContext: PlannerContext =
     new PlannerContext(
@@ -148,14 +144,7 @@ abstract class PlannerBase(
   def getFlinkContext: FlinkContext = plannerContext.getFlinkContext
 
   /**
-   * Gives access to both API specific table configuration and executor configuration.
-   *
-   * This configuration should be the main source of truth in the planner module.
-   */
-  def getConfiguration: ReadableConfig = plannerConfiguration
-
-  /**
-   * @deprecated Do not use this method anymore. Use [[getConfiguration]] to access options.
+   * @deprecated Do not use this method anymore. Use [[getTableConfig]] to access options.
    *             Create transformations without it. A [[StreamExecutionEnvironment]] is a mixture
    *             of executor and stream graph generator/builder. In the long term, we would like
    *             to avoid the need for it in the planner module.
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala
index c3839bc..1c4a512 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/delegation/StreamPlanner.scala
@@ -211,7 +211,7 @@ class StreamPlanner(
 
   override def beforeTranslation(): Unit = {
     super.beforeTranslation()
-    val runtimeMode = getConfiguration.get(ExecutionOptions.RUNTIME_MODE)
+    val runtimeMode = getTableConfig.get(ExecutionOptions.RUNTIME_MODE)
     if (runtimeMode != RuntimeExecutionMode.STREAMING) {
       throw new IllegalArgumentException(
         "Mismatch between configured runtime mode and actual runtime mode. " +
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
index 62d2d9a..ba08b50 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
@@ -41,7 +41,7 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner)
 
   override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = {
     // build RelNodeBlock plan
-    val rootBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, planner.getConfiguration)
+    val rootBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, planner.getTableConfig)
     // optimize recursively RelNodeBlock
     rootBlocks.foreach(optimizeBlock)
     rootBlocks
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
index 6205d93..a449858 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
@@ -49,17 +49,17 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner)
   extends CommonSubGraphBasedOptimizer {
 
   override protected def doOptimize(roots: Seq[RelNode]): Seq[RelNodeBlock] = {
-    val config = planner.getConfiguration
+    val tableConfig = planner.getTableConfig
     // build RelNodeBlock plan
-    val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, config)
+    val sinkBlocks = RelNodeBlockPlanBuilder.buildRelNodeBlockPlan(roots, tableConfig)
     // infer trait properties for sink block
     sinkBlocks.foreach { sinkBlock =>
       // don't require update before by default
       sinkBlock.setUpdateBeforeRequired(false)
 
-      val miniBatchInterval: MiniBatchInterval = if (config.get(
+      val miniBatchInterval: MiniBatchInterval = if (tableConfig.get(
         ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED)) {
-        val miniBatchLatency = config.get(
+        val miniBatchLatency = tableConfig.get(
           ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY).toMillis
         Preconditions.checkArgument(miniBatchLatency > 0,
           "MiniBatch Latency must be greater than 0 ms.", null)
@@ -156,7 +156,7 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner)
     val config = planner.getTableConfig
     val calciteConfig = TableConfigUtils.getCalciteConfig(config)
     val programs = calciteConfig.getStreamProgram
-      .getOrElse(FlinkStreamProgram.buildProgram(config.getConfiguration))
+      .getOrElse(FlinkStreamProgram.buildProgram(config))
     Preconditions.checkNotNull(programs)
 
     val context = relNode.getCluster.getPlanner.getContext.unwrap(classOf[FlinkContext])
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala
index ef9db28..b1a1265 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/metadata/FlinkRelMdHandlerTestBase.scala
@@ -978,7 +978,7 @@ class FlinkRelMdHandlerTestBase {
       cluster, streamPhysicalTraits, streamTs, program, program.getOutputRowType)
     val streamExchange = new StreamPhysicalExchange(
       cluster, streamPhysicalTraits.replace(hash01), streamCalc, hash01)
-    val emitStrategy = WindowEmitStrategy(tableConfig.getConfiguration, tumblingGroupWindow)
+    val emitStrategy = WindowEmitStrategy(tableConfig, tumblingGroupWindow)
     val streamWindowAgg = new StreamPhysicalGroupWindowTableAggregate(
       cluster,
       streamPhysicalTraits,
@@ -1620,7 +1620,7 @@ class FlinkRelMdHandlerTestBase {
       cluster, streamPhysicalTraits, streamTs, program, program.getOutputRowType)
     val streamExchange = new StreamPhysicalExchange(
       cluster, streamPhysicalTraits.replace(hash01), streamCalc, hash01)
-    val emitStrategy = WindowEmitStrategy(tableConfig.getConfiguration, tumblingGroupWindow)
+    val emitStrategy = WindowEmitStrategy(tableConfig, tumblingGroupWindow)
     val streamWindowAgg = new StreamPhysicalGroupWindowAggregate(
       cluster,
       streamPhysicalTraits,
@@ -1759,7 +1759,7 @@ class FlinkRelMdHandlerTestBase {
       cluster, streamPhysicalTraits, streamTs, program, program.getOutputRowType)
     val streamExchange = new StreamPhysicalExchange(
       cluster, streamPhysicalTraits.replace(hash1), streamCalc, hash1)
-    val emitStrategy = WindowEmitStrategy(tableConfig.getConfiguration, tumblingGroupWindow)
+    val emitStrategy = WindowEmitStrategy(tableConfig, tumblingGroupWindow)
     val streamWindowAgg = new StreamPhysicalGroupWindowAggregate(
       cluster,
       streamPhysicalTraits,
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala
index 9d93773..bfbae11 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalHashAggRuleTest.scala
@@ -38,7 +38,7 @@ class EnforceLocalHashAggRuleTest extends EnforceLocalAggRuleTestBase {
   @Before
   override def setup(): Unit = {
     super.setup()
-    val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig.getConfiguration)
+    val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig)
     // remove the original BatchExecHashAggRule and add BatchExecHashAggRuleForOnePhase
     // to let the physical phase generate one phase aggregate
     program.getFlinkRuleSetProgram(FlinkBatchProgram.PHYSICAL)
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala
index 29520c5..03a72a1 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/batch/EnforceLocalSortAggRuleTest.scala
@@ -42,7 +42,7 @@ class EnforceLocalSortAggRuleTest extends EnforceLocalAggRuleTestBase {
     super.setup()
     util.addFunction("weightedAvg", new WeightedAvg)
 
-    val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig.getConfiguration)
+    val program = FlinkBatchProgram.buildProgram(util.tableEnv.getConfig)
     // remove the original BatchExecSortAggRule and add BatchExecSortAggRuleForOnePhase
     // to let the physical phase generate one phase aggregate
     program.getFlinkRuleSetProgram(FlinkBatchProgram.PHYSICAL)
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
index c125235..1668880 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
@@ -43,7 +43,7 @@ class AggregateJoinTransposeITCase extends BatchTestBase {
   @Before
   override def before(): Unit = {
     super.before()
-    val programs = FlinkBatchProgram.buildProgram(tEnv.getConfig.getConfiguration)
+    val programs = FlinkBatchProgram.buildProgram(tEnv.getConfig)
     // remove FlinkAggregateJoinTransposeRule from logical program (volcano planner)
     programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL)
       .getOrElse(throw new TableException(s"${FlinkBatchProgram.LOGICAL} does not exist"))
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
index cb56447..83818a5 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
@@ -32,7 +32,6 @@ import org.apache.flink.streaming.api.environment.{LocalStreamEnvironment, Strea
 import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecEnv}
 import org.apache.flink.streaming.api.{TimeCharacteristic, environment}
 import org.apache.flink.table.api._
-import org.apache.flink.table.api.bridge.internal.AbstractStreamTableEnvironmentImpl
 import org.apache.flink.table.api.bridge.java.{StreamTableEnvironment => JavaStreamTableEnv}
 import org.apache.flink.table.api.bridge.scala.{StreamTableEnvironment => ScalaStreamTableEnv}
 import org.apache.flink.table.api.config.ExecutionConfigOptions
@@ -1261,7 +1260,7 @@ case class StreamTableTestUtil(
   }
 
   def buildStreamProgram(firstProgramNameToRemove: String): Unit = {
-    val program = FlinkStreamProgram.buildProgram(tableEnv.getConfig.getConfiguration)
+    val program = FlinkStreamProgram.buildProgram(tableEnv.getConfig)
     var startRemove = false
     program.getProgramNames.foreach {
       name =>
@@ -1285,8 +1284,7 @@ case class StreamTableTestUtil(
   def getStreamProgram(): FlinkChainedProgram[StreamOptimizeContext] = {
     val tableConfig = tableEnv.getConfig
     val calciteConfig = TableConfigUtils.getCalciteConfig(tableConfig)
-    calciteConfig.getStreamProgram.getOrElse(FlinkStreamProgram.buildProgram(
-      tableConfig.getConfiguration))
+    calciteConfig.getStreamProgram.getOrElse(FlinkStreamProgram.buildProgram(tableConfig))
   }
 
   def enableMiniBatch(): Unit = {
@@ -1346,7 +1344,7 @@ case class BatchTableTestUtil(
   extends TableTestUtil(test, isStreamingMode = false, catalogManager, conf) {
 
   def buildBatchProgram(firstProgramNameToRemove: String): Unit = {
-    val program = FlinkBatchProgram.buildProgram(tableEnv.getConfig.getConfiguration)
+    val program = FlinkBatchProgram.buildProgram(tableEnv.getConfig)
     var startRemove = false
     program.getProgramNames.foreach {
       name =>
@@ -1370,8 +1368,7 @@ case class BatchTableTestUtil(
   def getBatchProgram(): FlinkChainedProgram[BatchOptimizeContext] = {
     val tableConfig = tableEnv.getConfig
     val calciteConfig = TableConfigUtils.getCalciteConfig(tableConfig)
-    calciteConfig.getBatchProgram.getOrElse(FlinkBatchProgram.buildProgram(
-      tableConfig.getConfiguration))
+    calciteConfig.getBatchProgram.getOrElse(FlinkBatchProgram.buildProgram(tableConfig))
   }
 
   def createCollectTableSink(

[flink] 08/08: [hotfix][table-planner] Remove ExecNodeConfig#getMaxIdleStateRetentionTime()

Posted by tw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

twalthr pushed a commit to branch release-1.15
in repository https://gitbox.apache.org/repos/asf/flink.git

commit 65a3b0807d691b7738351fd81dab1b4418af2448
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Mon Mar 21 19:53:47 2022 +0200

    [hotfix][table-planner] Remove ExecNodeConfig#getMaxIdleStateRetentionTime()
    
    Replace `ExecNodeConfig#getMaxIdleStateRetentionTime()` with a method in
    `TableConfigUtils` to keep the class clean and remove another direct
    usages of `TableConfig` in favour of `ReadableConfig`.
    
    (cherry picked from commit 237cec31e6d8b70c2b9a0f939c7a8a2b04c24568)
---
 .../flink/table/planner/plan/nodes/exec/ExecNodeConfig.java   | 11 -----------
 .../plan/nodes/exec/stream/StreamExecOverAggregate.java       | 11 ++++++-----
 .../nodes/exec/stream/StreamExecPythonGroupAggregate.java     |  3 ++-
 .../exec/stream/StreamExecPythonGroupTableAggregate.java      |  3 ++-
 .../plan/nodes/exec/stream/StreamExecPythonOverAggregate.java |  3 ++-
 .../plan/nodes/exec/stream/StreamExecTemporalJoin.java        |  3 ++-
 .../apache/flink/table/planner/utils/TableConfigUtils.java    | 11 +++++++++++
 7 files changed, 25 insertions(+), 20 deletions(-)

diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
index 13df31b..2a3e0a0 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java
@@ -82,15 +82,4 @@ public final class ExecNodeConfig implements ReadableConfig {
     public long getStateRetentionTime() {
         return get(ExecutionConfigOptions.IDLE_STATE_RETENTION).toMillis();
     }
-
-    // See https://issues.apache.org/jira/browse/FLINK-26190
-    /**
-     * Using {@link #tableConfig} to satisify tests like {@code OverAggregateHarnessTest}, which use
-     * {@code HarnessTestBase#TestTableConfig} to individually manipulate the
-     * maxIdleStateRetentionTime. See {@link TableConfig#getMaxIdleStateRetentionTime()}.
-     */
-    @Deprecated
-    public long getMaxIdleStateRetentionTime() {
-        return tableConfig.getMaxIdleStateRetentionTime();
-    }
 }
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
index 0a189f98..1eb874c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
@@ -44,6 +44,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedAggsHandleFunction;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.operators.over.ProcTimeRangeBoundedPrecedingFunction;
@@ -307,7 +308,7 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
                 // ROWS unbounded over process function
                 return new RowTimeRowsUnboundedPrecedingFunction<>(
                         config.getStateRetentionTime(),
-                        config.getMaxIdleStateRetentionTime(),
+                        TableConfigUtils.getMaxIdleStateRetentionTime(config),
                         genAggsHandler,
                         flattenAccTypes,
                         fieldTypes,
@@ -316,7 +317,7 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
                 // RANGE unbounded over process function
                 return new RowTimeRangeUnboundedPrecedingFunction<>(
                         config.getStateRetentionTime(),
-                        config.getMaxIdleStateRetentionTime(),
+                        TableConfigUtils.getMaxIdleStateRetentionTime(config),
                         genAggsHandler,
                         flattenAccTypes,
                         fieldTypes,
@@ -325,7 +326,7 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
         } else {
             return new ProcTimeUnboundedPrecedingFunction<>(
                     config.getStateRetentionTime(),
-                    config.getMaxIdleStateRetentionTime(),
+                    TableConfigUtils.getMaxIdleStateRetentionTime(config),
                     genAggsHandler,
                     flattenAccTypes);
         }
@@ -393,7 +394,7 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
             if (isRowsClause) {
                 return new RowTimeRowsBoundedPrecedingFunction<>(
                         config.getStateRetentionTime(),
-                        config.getMaxIdleStateRetentionTime(),
+                        TableConfigUtils.getMaxIdleStateRetentionTime(config),
                         genAggsHandler,
                         flattenAccTypes,
                         fieldTypes,
@@ -407,7 +408,7 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
             if (isRowsClause) {
                 return new ProcTimeRowsBoundedPrecedingFunction<>(
                         config.getStateRetentionTime(),
-                        config.getMaxIdleStateRetentionTime(),
+                        TableConfigUtils.getMaxIdleStateRetentionTime(config),
                         genAggsHandler,
                         flattenAccTypes,
                         fieldTypes,
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
index d346e47..0a977a6 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
@@ -39,6 +39,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.dataview.DataViewSpec;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -160,7 +161,7 @@ public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase {
                         pythonFunctionInfos,
                         dataViewSpecs,
                         config.getStateRetentionTime(),
-                        config.getMaxIdleStateRetentionTime(),
+                        TableConfigUtils.getMaxIdleStateRetentionTime(config),
                         inputCountIndex,
                         countStarInserted);
         // partitioned aggregation
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
index 40a1cf4..583fafc 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
@@ -41,6 +41,7 @@ import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
 import org.apache.flink.table.planner.plan.utils.AggregateUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.dataview.DataViewSpec;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -134,7 +135,7 @@ public class StreamExecPythonGroupTableAggregate extends ExecNodeBase<RowData>
                         pythonFunctionInfos,
                         dataViewSpecs,
                         config.getStateRetentionTime(),
-                        config.getMaxIdleStateRetentionTime(),
+                        TableConfigUtils.getMaxIdleStateRetentionTime(config),
                         generateUpdateBefore,
                         inputCountIndex);
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
index efc844b..536a26a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
@@ -43,6 +43,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
 import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.plan.utils.OverAggregateUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedProjection;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
@@ -187,7 +188,7 @@ public class StreamExecPythonOverAggregate extends ExecNodeBase<RowData>
                         precedingOffset,
                         group.isRows(),
                         config.getStateRetentionTime(),
-                        config.getMaxIdleStateRetentionTime(),
+                        TableConfigUtils.getMaxIdleStateRetentionTime(config),
                         pythonConfig,
                         config);
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
index 11125be..8b32e1a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
@@ -43,6 +43,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
 import org.apache.flink.table.planner.plan.utils.JoinUtil;
 import org.apache.flink.table.planner.plan.utils.KeySelectorUtil;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
+import org.apache.flink.table.planner.utils.TableConfigUtils;
 import org.apache.flink.table.runtime.generated.GeneratedJoinCondition;
 import org.apache.flink.table.runtime.keyselector.RowDataKeySelector;
 import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
@@ -248,7 +249,7 @@ public class StreamExecTemporalJoin extends ExecNodeBase<RowData>
 
         boolean isLeftOuterJoin = joinSpec.getJoinType() == FlinkJoinType.LEFT;
         long minRetentionTime = config.getStateRetentionTime();
-        long maxRetentionTime = config.getMaxIdleStateRetentionTime();
+        long maxRetentionTime = TableConfigUtils.getMaxIdleStateRetentionTime(config);
         if (rightTimeAttributeIndex >= 0) {
             return new TemporalRowTimeJoinOperator(
                     InternalTypeInfo.of(leftInputType),
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
index 02f00ca..7c22ac0 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/TableConfigUtils.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.utils;
 
 import org.apache.flink.configuration.ReadableConfig;
 import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.api.config.TableConfigOptions;
 import org.apache.flink.table.planner.calcite.CalciteConfig;
 import org.apache.flink.table.planner.calcite.CalciteConfig$;
@@ -107,6 +108,16 @@ public class TableConfigUtils {
                 : ZoneId.of(zone);
     }
 
+    /**
+     * Similar to {@link TableConfig#getMaxIdleStateRetentionTime()}.
+     *
+     * @see TableConfig#getMaxIdleStateRetentionTime()
+     */
+    @Deprecated
+    public static long getMaxIdleStateRetentionTime(ReadableConfig tableConfig) {
+        return tableConfig.get(ExecutionConfigOptions.IDLE_STATE_RETENTION).toMillis() * 3 / 2;
+    }
+
     /** Validates user configured time zone. */
     private static void validateTimeZone(String zone) {
         final String zoneId = zone.toUpperCase();