You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ku...@apache.org on 2019/07/11 05:38:26 UTC

[flink] branch master updated: [FLINK-12348][table-planner-blink] Use TableConfig in api module to replace TableConfig in blink-planner module.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 428c704  [FLINK-12348][table-planner-blink] Use TableConfig in api module to replace TableConfig in blink-planner module.
428c704 is described below

commit 428c7049882cfc482562b69d1a367cdb9b0eec72
Author: beyond1920 <be...@126.com>
AuthorDate: Wed Jul 10 13:39:19 2019 +0800

    [FLINK-12348][table-planner-blink] Use TableConfig in api module to replace TableConfig in blink-planner module.
    
    This closes #8294
---
 flink-python/pyflink/table/table_config.py         |   9 +
 .../table/tests/test_table_config_completeness.py  |   2 +-
 .../org/apache/flink/table/api/TableConfig.java    |  33 ++
 .../resource/parallelism/ParallelismProcessor.java |   2 +-
 .../apache/flink/table/planner/PlannerContext.java |   3 +-
 .../flink/table/util/AggregatePhaseStrategy.java}  |  29 +-
 .../apache/flink/table/util/TableConfigUtils.java  | 118 ++++++++
 .../org/apache/flink/table/api/TableConfig.scala   | 331 ---------------------
 .../apache/flink/table/calcite/CalciteConfig.scala |   3 +-
 .../flink/table/codegen/ExpressionReducer.scala    |   8 +-
 .../codegen/NestedLoopJoinCodeGenerator.scala      |   2 +-
 .../plan/metadata/FlinkRelMdColumnNullCount.scala  |   2 +-
 .../plan/metadata/FlinkRelMdDistinctRowCount.scala |   2 +-
 .../plan/metadata/FlinkRelMdDistribution.scala     |   2 +-
 .../table/plan/metadata/FlinkRelMdRowCount.scala   |   3 +-
 .../table/plan/metadata/SelectivityEstimator.scala |   2 +-
 .../table/plan/nodes/common/CommonLookupJoin.scala |  16 +-
 .../plan/nodes/logical/FlinkLogicalSort.scala      |   8 +-
 .../batch/BatchExecGroupAggregateBase.scala        |   8 +-
 .../physical/batch/BatchExecHashAggregate.scala    |   2 +-
 .../batch/BatchExecHashAggregateBase.scala         |   2 +-
 .../nodes/physical/batch/BatchExecHashJoin.scala   |   2 +-
 .../batch/BatchExecHashWindowAggregateBase.scala   |   4 +-
 .../physical/batch/BatchExecOverAggregate.scala    |   4 +-
 .../plan/nodes/physical/batch/BatchExecRank.scala  |   2 +-
 .../plan/nodes/physical/batch/BatchExecSink.scala  |   2 +-
 .../plan/nodes/physical/batch/BatchExecSort.scala  |   2 +-
 .../physical/batch/BatchExecSortAggregate.scala    |   2 +-
 .../physical/batch/BatchExecSortMergeJoin.scala    |   4 +-
 .../batch/BatchExecSortWindowAggregateBase.scala   |   2 +-
 .../physical/stream/StreamExecDeduplicate.scala    |   2 +-
 .../stream/StreamExecGlobalGroupAggregate.scala    |   2 +-
 .../physical/stream/StreamExecGroupAggregate.scala |   2 +-
 .../stream/StreamExecGroupWindowAggregate.scala    |   4 -
 .../nodes/physical/stream/StreamExecLimit.scala    |   2 +-
 .../nodes/physical/stream/StreamExecRank.scala     |   2 +-
 .../nodes/physical/stream/StreamExecSink.scala     |   2 +-
 .../nodes/physical/stream/StreamExecSort.scala     |   2 +-
 .../physical/stream/StreamExecSortLimit.scala      |   2 +-
 .../stream/StreamExecWatermarkAssigner.scala       |   5 +-
 .../BatchCommonSubGraphBasedOptimizer.scala        |   5 +-
 .../flink/table/plan/optimize/RelNodeBlock.scala   |   4 +-
 .../StreamCommonSubGraphBasedOptimizer.scala       |  11 +-
 .../flink/table/plan/reuse/SubplanReuser.scala     |   4 +-
 .../rules/logical/JoinDeriveNullFilterRule.scala   |   2 +-
 .../PushFilterIntoTableSourceScanRule.scala        |   2 +-
 .../plan/rules/logical/SplitAggregateRule.scala    |   6 +-
 .../physical/batch/BatchExecAggRuleBase.scala      |  28 +-
 .../physical/batch/BatchExecHashAggRule.scala      |   7 +-
 .../physical/batch/BatchExecHashJoinRule.scala     |  13 +-
 .../physical/batch/BatchExecJoinRuleBase.scala     |   2 +-
 .../batch/BatchExecNestedLoopJoinRule.scala        |   5 +-
 .../physical/batch/BatchExecSortAggRule.scala      |   7 +-
 .../batch/BatchExecSortMergeJoinRule.scala         |  10 +-
 .../rules/physical/batch/BatchExecSortRule.scala   |   3 +-
 .../physical/stream/IncrementalAggregateRule.scala |   2 +-
 .../stream/MiniBatchIntervalInferRule.scala        |   2 +-
 .../stream/TwoStageOptimizedAggregateRule.scala    |  10 +-
 .../flink/table/plan/util/AggregateUtil.scala      |   2 +-
 .../flink/table/plan/util/FlinkRelOptUtil.scala    |   2 +-
 .../apache/flink/table/plan/util/RankUtil.scala    |   4 +-
 .../flink/table/plan/util/WindowEmitStrategy.scala | 127 ++++++--
 .../flink/table/plan/util/WindowJoinUtil.scala     |   2 +-
 .../apache/flink/table/planner/PlannerBase.scala   |   4 +-
 .../apache/flink/table/api/batch/ExplainTest.scala |   2 +-
 .../flink/table/api/stream/ExplainTest.scala       |   4 +-
 .../table/calcite/CalciteConfigBuilderTest.scala   |   4 +-
 .../table/plan/batch/sql/DagOptimizationTest.scala |  24 +-
 .../table/plan/batch/sql/DeadlockBreakupTest.scala |  46 +--
 .../table/plan/batch/sql/RemoveCollationTest.scala |  41 +--
 .../table/plan/batch/sql/RemoveShuffleTest.scala   | 114 +++----
 .../table/plan/batch/sql/SetOperatorsTest.scala    |   2 +-
 .../flink/table/plan/batch/sql/SinkTest.scala      |   2 +-
 .../flink/table/plan/batch/sql/SortLimitTest.scala |  29 +-
 .../flink/table/plan/batch/sql/SortTest.scala      |  27 +-
 .../table/plan/batch/sql/SubplanReuseTest.scala    |  50 ++--
 .../plan/batch/sql/agg/HashAggregateTest.scala     |  21 +-
 .../plan/batch/sql/agg/SortAggregateTest.scala     |  21 +-
 .../plan/batch/sql/agg/WindowAggregateTest.scala   |  19 +-
 .../batch/sql/join/BroadcastHashJoinTest.scala     |   4 +-
 .../sql/join/BroadcastHashSemiAntiJoinTest.scala   |   4 +-
 .../table/plan/batch/sql/join/LookupJoinTest.scala |  15 +-
 .../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 +-
 .../common/AggregateReduceGroupingTestBase.scala   |  12 +-
 .../table/plan/common/JoinReorderTestBase.scala    |   2 +-
 .../plan/metadata/SelectivityEstimatorTest.scala   |   2 +-
 .../plan/nodes/resource/ExecNodeResourceTest.scala |  10 +-
 .../logical/AggregateReduceGroupingRuleTest.scala  |   8 +-
 .../logical/CalcPruneAggregateCallRuleTest.scala   |   9 +-
 .../rules/logical/ConvertToNotInOrInRuleTest.scala |   5 +-
 .../FlinkAggregateInnerJoinTransposeRuleTest.scala |   5 +-
 .../FlinkAggregateOuterJoinTransposeRuleTest.scala |   5 +-
 .../logical/FlinkAggregateRemoveRuleTest.scala     |   5 +-
 .../rules/logical/FlinkCalcMergeRuleTest.scala     |   5 +-
 .../rules/logical/FlinkFilterJoinRuleTest.scala    |   5 +-
 .../logical/FlinkJoinPushExpressionsRuleTest.scala |   5 +-
 .../rules/logical/FlinkLimit0RemoveRuleTest.scala  |   5 +-
 .../rules/logical/FlinkPruneEmptyRulesTest.scala   |   5 +-
 .../FlinkSemiAntiJoinFilterTransposeRuleTest.scala |   5 +-
 .../FlinkSemiAntiJoinJoinTransposeRuleTest.scala   |   5 +-
 ...FlinkSemiAntiJoinProjectTransposeRuleTest.scala |   5 +-
 .../JoinConditionEqualityTransferRuleTest.scala    |   5 +-
 .../logical/JoinConditionTypeCoerceRuleTest.scala  |   5 +-
 .../JoinDependentConditionDerivationRuleTest.scala |   5 +-
 .../logical/JoinDeriveNullFilterRuleTest.scala     |   7 +-
 .../plan/rules/logical/LogicalUnnestRuleTest.scala |   7 +-
 .../ProjectPruneAggregateCallRuleTest.scala        |   9 +-
 .../ProjectSemiAntiJoinTransposeRuleTest.scala     |   5 +-
 .../PushFilterIntoTableSourceScanRuleTest.scala    |   5 +-
 .../PushProjectIntoTableSourceScanRuleTest.scala   |   5 +-
 .../ReplaceIntersectWithSemiJoinRuleTest.scala     |   6 +-
 .../logical/ReplaceMinusWithAntiJoinRuleTest.scala |   6 +-
 .../rules/logical/RewriteCoalesceRuleTest.scala    |   6 +-
 .../logical/RewriteIntersectAllRuleTest.scala      |   6 +-
 .../rules/logical/RewriteMinusAllRuleTest.scala    |   6 +-
 .../RewriteMultiJoinConditionRuleTest.scala        |   5 +-
 .../logical/SimplifyFilterConditionRuleTest.scala  |   5 +-
 .../logical/SimplifyJoinConditionRuleTest.scala    |   5 +-
 .../rules/logical/SplitAggregateRuleTest.scala     |   4 +-
 .../rules/logical/subquery/SubQueryTestBase.scala  |   5 +-
 .../RemoveRedundantLocalHashAggRuleTest.scala      |   8 +-
 .../RemoveRedundantLocalSortAggRuleTest.scala      |   6 +-
 .../RetractionRulesWithTwoStageAggTest.scala       |  11 +-
 .../plan/stream/sql/DagOptimizationTest.scala      |  26 +-
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  47 ++-
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  12 +-
 .../table/plan/stream/sql/SubplanReuseTest.scala   |  14 +-
 .../table/plan/stream/sql/agg/AggregateTest.scala  |  16 +-
 .../stream/sql/agg/DistinctAggregateTest.scala     |  23 +-
 .../stream/sql/agg/IncrementalAggregateTest.scala  |   9 +-
 .../stream/sql/agg/TwoStageAggregateTest.scala     |  11 +-
 .../flink/table/runtime/batch/sql/CalcITCase.scala |   4 +-
 .../table/runtime/batch/sql/DecimalITCase.scala    |  14 +-
 .../flink/table/runtime/batch/sql/MiscITCase.scala |   4 +-
 .../table/runtime/batch/sql/SortLimitITCase.scala  |   2 +-
 .../table/runtime/batch/sql/UnionITCase.scala      |   6 +-
 .../sql/agg/AggregateJoinTransposeITCase.scala     |  13 +-
 .../sql/agg/AggregateReduceGroupingITCase.scala    |  21 +-
 .../runtime/batch/sql/agg/HashAggITCase.scala      |   2 +-
 .../sql/agg/HashDistinctAggregateITCase.scala      |   5 +-
 .../runtime/batch/sql/agg/SortAggITCase.scala      |  11 +-
 .../sql/agg/SortDistinctAggregateITCase.scala      |   5 +-
 .../runtime/batch/sql/join/InnerJoinITCase.scala   |  10 +-
 .../table/runtime/batch/sql/join/JoinITCase.scala  |   2 +-
 .../runtime/batch/sql/join/JoinITCaseHelper.scala  |   4 +-
 .../runtime/harness/OverWindowHarnessTest.scala    |  67 +++--
 .../runtime/stream/sql/OverWindowITCase.scala      |   3 -
 .../table/runtime/stream/sql/SortITCase.scala      |  18 +-
 .../runtime/stream/sql/SplitAggregateITCase.scala  |   4 +-
 .../runtime/stream/sql/WindowAggregateITCase.scala |  22 +-
 .../runtime/stream/table/AggregateITCase.scala     |   2 +-
 .../table/runtime/stream/table/JoinITCase.scala    |   4 +-
 .../flink/table/runtime/utils/BatchTestBase.scala  |  10 +-
 .../runtime/utils/StreamingWithAggTestBase.scala   |  14 +-
 .../utils/StreamingWithMiniBatchTestBase.scala     |  12 +-
 .../apache/flink/table/util/TableTestBase.scala    |  40 ++-
 161 files changed, 991 insertions(+), 1042 deletions(-)

diff --git a/flink-python/pyflink/table/table_config.py b/flink-python/pyflink/table/table_config.py
index 8e0fbec..399778c 100644
--- a/flink-python/pyflink/table/table_config.py
+++ b/flink-python/pyflink/table/table_config.py
@@ -38,6 +38,15 @@ class TableConfig(object):
         else:
             self._j_table_config = j_table_config
 
+    def set_conf(self, key, value):
+        """
+        Adds the given key/value pair.
+
+        :param key: the key of the key/value pair to be added.
+        :param value: the value of the key/value pair to be added.
+        """
+        self._j_table_config.setConf(key, value)
+
     def get_timezone(self):
         """
         Returns the timezone id, either an abbreviation such as "PST", a full name such as
diff --git a/flink-python/pyflink/table/tests/test_table_config_completeness.py b/flink-python/pyflink/table/tests/test_table_config_completeness.py
index 9db60cf..b71546d 100644
--- a/flink-python/pyflink/table/tests/test_table_config_completeness.py
+++ b/flink-python/pyflink/table/tests/test_table_config_completeness.py
@@ -39,7 +39,7 @@ class TableConfigCompletenessTests(PythonAPICompletenessTestCase, unittest.TestC
     @classmethod
     def excluded_methods(cls):
         # internal interfaces, no need to expose to users.
-        return {'getPlannerConfig', 'setPlannerConfig'}
+        return {'getPlannerConfig', 'setPlannerConfig', 'addConfiguration', 'getConfiguration'}
 
     @classmethod
     def java_method_name(cls, python_method_name):
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
index 6ee7f8d..c10fbb7 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableConfig.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.api;
 
 import org.apache.flink.annotation.PublicEvolving;
 import org.apache.flink.api.common.time.Time;
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.util.Preconditions;
 
 import java.math.MathContext;
@@ -71,6 +72,38 @@ public class TableConfig {
 	private long maxIdleStateRetentionTime = 0L;
 
 	/**
+	 * A configuration object to hold all key/value configuration.
+	 */
+	private Configuration configuration = new Configuration();
+
+	/**
+	 * Returns all key/value configuration.
+	 */
+	public Configuration getConfiguration() {
+		return configuration;
+	}
+
+	/**
+	 * Adds the given key/value configuration.
+	 *
+	 * @param configuration key/value configuration to adds
+	 */
+	public void addConfiguration(Configuration configuration) {
+		Preconditions.checkNotNull(configuration);
+		this.configuration.addAll(configuration);
+	}
+
+	/**
+	 * Adds the given key/value pair.
+	 *
+	 * @param key the key of the key/value pair to be added
+	 * @param value the value of the key/value pair to be added
+	 */
+	public void setConf(String key, String value) {
+		configuration.setString(key, value);
+	}
+
+	/**
 	 * Returns the timezone for date/time/timestamp conversions.
 	 */
 	public TimeZone getTimeZone() {
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ParallelismProcessor.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ParallelismProcessor.java
index c24ce30..163b712 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ParallelismProcessor.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/plan/nodes/resource/parallelism/ParallelismProcessor.java
@@ -47,7 +47,7 @@ public class ParallelismProcessor implements DAGProcessor {
 			ShuffleStageGenerator.generate(rootNodes, nodeToFinalParallelismMap);
 		// calculate parallelism of shuffleStages.
 		ShuffleStageParallelismCalculator.calculate(
-			planner.getTableConfig().getConf(), planner.getExecEnv().getParallelism(), nodeShuffleStageMap.values());
+			planner.getTableConfig().getConfiguration(), planner.getExecEnv().getParallelism(), nodeShuffleStageMap.values());
 		for (ExecNode<?, ?> node : nodeShuffleStageMap.keySet()) {
 			node.getResource().setParallelism(nodeShuffleStageMap.get(node).getParallelism());
 		}
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/PlannerContext.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/PlannerContext.java
index 5bce41b..c323ea0 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/PlannerContext.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/PlannerContext.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.codegen.ExpressionReducer;
 import org.apache.flink.table.functions.sql.FlinkSqlOperatorTable;
 import org.apache.flink.table.plan.cost.FlinkCostFactory;
 import org.apache.flink.table.util.JavaScalaConversionUtil;
+import org.apache.flink.table.util.TableConfigUtils;
 
 import org.apache.calcite.config.Lex;
 import org.apache.calcite.jdbc.CalciteSchema;
@@ -172,7 +173,7 @@ public class PlannerContext {
 	}
 
 	private CalciteConfig getCalciteConfig(TableConfig tableConfig) {
-		return tableConfig.getCalciteConfig();
+		return TableConfigUtils.getCalciteConfig(tableConfig);
 	}
 
 	/**
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java
similarity index 53%
copy from flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
copy to flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java
index 306a1db..315a7f3 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/AggregatePhaseStrategy.java
@@ -16,17 +16,28 @@
  * limitations under the License.
  */
 
-package org.apache.flink.table.runtime.batch.sql.agg
+package org.apache.flink.table.util;
 
-import org.apache.flink.table.api.{OperatorType, ExecutionConfigOptions}
+import org.apache.flink.table.api.OptimizerConfigOptions;
 
 /**
-  * HashDistinctAggITCase using HashAgg Operator.
-  */
-class HashDistinctAggregateITCase extends DistinctAggregateITCaseBase {
+ * Aggregate phase strategy which could be specified in {@link OptimizerConfigOptions#SQL_OPTIMIZER_AGG_PHASE_STRATEGY}.
+ */
+public enum AggregatePhaseStrategy {
+
+	/**
+	 * No special enforcer for aggregate stage. Whether to choose two stage aggregate or one stage aggregate depends on cost.
+	 */
+	AUTO,
+
+	/**
+	 * Enforce to use one stage aggregate which only has CompleteGlobalAggregate.
+	 */
+	ONE_PHASE,
 
-  override def prepareAggOp(): Unit = {
-    tEnv.getConfig.getConf.setString(
-      ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
-  }
+	/**
+	 * Enforce to use two stage aggregate which has localAggregate and globalAggregate.
+	 * NOTE: If aggregate call does not support split into two phase, still use one stage aggregate.
+	 */
+	TWO_PHASE
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java
new file mode 100644
index 0000000..05abfe3
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/util/TableConfigUtils.java
@@ -0,0 +1,118 @@
+/*
+ * 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.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.table.api.TableConfig;
+import org.apache.flink.table.calcite.CalciteConfig;
+import org.apache.flink.table.calcite.CalciteConfig$;
+import org.apache.flink.table.plan.util.OperatorType;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import scala.concurrent.duration.Duration;
+
+import static org.apache.flink.table.api.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS;
+import static org.apache.flink.table.api.OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY;
+
+/**
+ * Utility class for {@link TableConfig} related helper functions.
+ */
+public class TableConfigUtils {
+
+	/**
+	 * Returns whether the given operator type is disabled.
+	 *
+	 * @param tableConfig TableConfig object
+	 * @param operatorType operator type to check
+	 * @return true if the given operator is disabled.
+	 */
+	public static boolean isOperatorDisabled(TableConfig tableConfig, OperatorType operatorType) {
+		String value = tableConfig.getConfiguration().getString(SQL_EXEC_DISABLED_OPERATORS);
+		String[] operators = value.split(",");
+		Set<OperatorType> operatorSets = new HashSet<>();
+		for (String operator : operators) {
+			operator = operator.trim();
+			if (operator.isEmpty()) {
+				continue;
+			}
+			if (operator.equals("HashJoin")) {
+				operatorSets.add(OperatorType.BroadcastHashJoin);
+				operatorSets.add(OperatorType.ShuffleHashJoin);
+			} else {
+				operatorSets.add(OperatorType.valueOf(operator));
+			}
+		}
+		return operatorSets.contains(operatorType);
+	}
+
+	/**
+	 * Returns the aggregate phase strategy configuration.
+	 *
+	 * @param tableConfig TableConfig object
+	 * @return the aggregate phase strategy
+	 */
+	public static AggregatePhaseStrategy getAggPhaseStrategy(TableConfig tableConfig) {
+		String aggPhaseConf = tableConfig.getConfiguration().getString(SQL_OPTIMIZER_AGG_PHASE_STRATEGY).trim();
+		if (aggPhaseConf.isEmpty()) {
+			return AggregatePhaseStrategy.AUTO;
+		} else {
+			return AggregatePhaseStrategy.valueOf(aggPhaseConf);
+		}
+	}
+
+	/**
+	 * Returns time in milli second.
+	 *
+	 * @param tableConfig TableConfig object
+	 * @param config config to fetch
+	 * @return time in milli second.
+	 */
+	public static Long getMillisecondFromConfigDuration(TableConfig tableConfig, ConfigOption<String> config) {
+		String timeStr = tableConfig.getConfiguration().getString(config);
+		if (timeStr != null) {
+			Duration duration = Duration.create(timeStr);
+			if (duration.isFinite()) {
+				return duration.toMillis();
+			} else {
+				throw new IllegalArgumentException(config.key() + " must be finite.");
+			}
+		} else {
+			return null;
+		}
+	}
+
+	/**
+	 * Returns {@link CalciteConfig} wraps in the given TableConfig.
+	 *
+	 * @param tableConfig TableConfig object
+	 * @return wrapped CalciteConfig.
+	 */
+	public static CalciteConfig getCalciteConfig(TableConfig tableConfig) {
+		return tableConfig.getPlannerConfig().unwrap(CalciteConfig.class).orElse(
+				CalciteConfig$.MODULE$.DEFAULT());
+	}
+
+	// Make sure that we cannot instantiate this class
+	private TableConfigUtils() {
+
+	}
+
+}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/TableConfig.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/TableConfig.scala
deleted file mode 100644
index 63a8d10..0000000
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/api/TableConfig.scala
+++ /dev/null
@@ -1,331 +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.api
-
-import org.apache.flink.api.common.time.Time
-import org.apache.flink.configuration.{ConfigOption, Configuration, GlobalConfiguration}
-import org.apache.flink.table.api.OperatorType.OperatorType
-import org.apache.flink.table.calcite.CalciteConfig
-import org.apache.flink.util.Preconditions
-
-import _root_.java.math.MathContext
-import _root_.java.util.TimeZone
-
-import _root_.scala.concurrent.duration.Duration
-
-/**
- * A config to define the runtime behavior of the Table API.
- */
-class TableConfig {
-
-  /**
-   * Defines the timezone for date/time/timestamp conversions.
-   */
-  private var timeZone: TimeZone = TimeZone.getTimeZone("UTC")
-
-  /**
-   * Defines if all fields need to be checked for NULL first.
-   */
-  private var nullCheck: Boolean = true
-
-  /**
-    * Defines the default context for decimal division calculation.
-    * We use Scala's default MathContext.DECIMAL128.
-    */
-  private var decimalContext: MathContext = MathContext.DECIMAL128
-
-  /**
-    * Specifies a threshold where generated code will be split into sub-function calls. Java has a
-    * maximum method length of 64 KB. This setting allows for finer granularity if necessary.
-    */
-  private var maxGeneratedCodeLength: Int = 64000 // just an estimate
-
-  /**
-    * The minimum time until state which was not updated will be retained.
-    * State might be cleared and removed if it was not updated for the defined period of time.
-    */
-  private var minIdleStateRetentionTime = 0L
-
-  /**
-    * The maximum time until state which was not updated will be retained.
-    * State will be cleared and removed if it was not updated for the defined period of time.
-    */
-  private var maxIdleStateRetentionTime = 0L
-
-  private val DEFAULT_FIRE_INTERVAL = Long.MinValue
-
-  /**
-    * The early firing interval in milli second, early fire is the emit strategy
-    * before watermark advanced to end of window.
-    *
-    * < 0 means no early fire
-    * 0 means no delay (fire on every element).
-    * > 0 means the fire interval
-    */
-  private var earlyFireInterval = DEFAULT_FIRE_INTERVAL
-
-  /**
-    * The late firing interval in milli second, late fire is the emit strategy
-    * after watermark advanced to end of window.
-    *
-    * < 0 means no late fire, drop every late elements
-    * 0 means no delay (fire on every element).
-    * > 0 means the fire interval
-    *
-    * NOTE: late firing strategy is only enabled when allowLateness > 0
-    */
-  private var lateFireInterval = DEFAULT_FIRE_INTERVAL
-
-  /**
-    * Defines the configuration of Calcite for Table API and SQL queries.
-    */
-  private var calciteConfig = CalciteConfig.createBuilder().build()
-
-  /**
-    * Defines user-defined configuration
-    */
-  private var conf = GlobalConfiguration.loadConfiguration()
-
-  /**
-   * Sets the timezone for date/time/timestamp conversions.
-   */
-  def setTimeZone(timeZone: TimeZone): Unit = {
-    require(timeZone != null, "timeZone must not be null.")
-    this.timeZone = timeZone
-  }
-
-  /**
-   * Returns the timezone for date/time/timestamp conversions.
-   */
-  def getTimeZone: TimeZone = timeZone
-
-  /**
-   * Returns the NULL check. If enabled, all fields need to be checked for NULL first.
-   */
-  def getNullCheck: Boolean = nullCheck
-
-  /**
-   * Sets the NULL check. If enabled, all fields need to be checked for NULL first.
-   */
-  def setNullCheck(nullCheck: Boolean): Unit = {
-    this.nullCheck = nullCheck
-  }
-
-  /**
-    * Returns the default context for decimal division calculation.
-    * [[_root_.java.math.MathContext#DECIMAL128]] by default.
-    */
-  def getDecimalContext: MathContext = decimalContext
-
-  /**
-    * Sets the default context for decimal division calculation.
-    * [[_root_.java.math.MathContext#DECIMAL128]] by default.
-    */
-  def setDecimalContext(mathContext: MathContext): Unit = {
-    this.decimalContext = mathContext
-  }
-
-  /**
-    * Returns the current threshold where generated code will be split into sub-function calls.
-    * Java has a maximum method length of 64 KB. This setting allows for finer granularity if
-    * necessary. Default is 64000.
-    */
-  def getMaxGeneratedCodeLength: Int = maxGeneratedCodeLength
-
-  /**
-    * Returns the current threshold where generated code will be split into sub-function calls.
-    * Java has a maximum method length of 64 KB. This setting allows for finer granularity if
-    * necessary. Default is 64000.
-    */
-  def setMaxGeneratedCodeLength(maxGeneratedCodeLength: Int): Unit = {
-    if (maxGeneratedCodeLength <= 0) {
-      throw new IllegalArgumentException("Length must be greater than 0.")
-    }
-    this.maxGeneratedCodeLength = maxGeneratedCodeLength
-  }
-
-  /**
-    * Returns user-defined configuration
-    */
-  def getConf: Configuration = conf
-
-  /**
-    * Sets user-defined configuration
-    */
-  def setConf(conf: Configuration): Unit = {
-    this.conf = GlobalConfiguration.loadConfiguration()
-    this.conf.addAll(conf)
-  }
-
-  /**
-    * Returns the current configuration of Calcite for Table API and SQL queries.
-    */
-  def getCalciteConfig: CalciteConfig = calciteConfig
-
-  /**
-    * Sets the configuration of Calcite for Table API and SQL queries.
-    * Changing the configuration has no effect after the first query has been defined.
-    */
-  def setCalciteConfig(calciteConfig: CalciteConfig): Unit = {
-    this.calciteConfig = Preconditions.checkNotNull(calciteConfig)
-  }
-
-  /**
-    * Returns true if given [[OperatorType]] is enabled, else false.
-    */
-  def isOperatorEnabled(operator: OperatorType): Boolean = {
-    val disableOperators = conf.getString(ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS)
-      .split(",")
-      .map(_.trim)
-    if (disableOperators.contains("HashJoin") &&
-      (operator == OperatorType.BroadcastHashJoin ||
-        operator == OperatorType.ShuffleHashJoin)) {
-      false
-    } else {
-      !disableOperators.contains(operator.toString)
-    }
-  }
-
-  def getMillisecondFromConfigDuration(config: ConfigOption[String]): Long = {
-    val duration = Duration.create(this.conf.getString(config))
-    if (duration.isFinite()) {
-      duration.toMillis
-    } else {
-      throw new IllegalArgumentException(config.key() + " must be finite.")
-    }
-  }
-
-  /**
-    * Specifies a minimum time interval for how long idle state, i.e., state which
-    * was not updated, will be retained.
-    * The maximum time will be 2 * minimum time if it > 0 or 1/2 * minimum time.
-    *
-    * State will never be cleared until it was idle for less than the minimum time and will never
-    * be kept if it was idle for more than the maximum time.
-    *
-    * When new data arrives for previously cleaned-up state, the new data will be handled as if it
-    * was the first data. This can result in previous results being overwritten.
-    *
-    * Set to 0 (zero) to never clean-up the state.
-    *
-    * @param minTime The minimum time interval for which idle state is retained. Set to 0 (zero) to
-    *                never clean-up the state.
-    */
-  def withIdleStateRetentionTime(minTime: Time): TableConfig = {
-    this.conf.setString(ExecutionConfigOptions.SQL_EXEC_STATE_TTL,
-      String.valueOf(minTime.toMilliseconds) + " ms")
-    this
-  }
-
-  /**
-    * Returns the minimum time until state which was not updated will be retained.
-    */
-  def getMinIdleStateRetentionTime: Long =
-    getMillisecondFromConfigDuration(ExecutionConfigOptions.SQL_EXEC_STATE_TTL)
-
-  /**
-    * Returns the maximum time until state which was not updated will be retained.
-    */
-  def getMaxIdleStateRetentionTime: Long = {
-    val ttlTime = getMinIdleStateRetentionTime
-    if (ttlTime >= 0) {
-      getMinIdleStateRetentionTime * 2
-    } else {
-      getMinIdleStateRetentionTime / 2
-    }
-  }
-
-  /**
-    * Specifies the early firing interval in milli second, early fire is the emit strategy
-    * before watermark advanced to end of window.
-    */
-  def withEarlyFireInterval(interval: Time): TableConfig = {
-    if (this.earlyFireInterval != DEFAULT_FIRE_INTERVAL
-      && this.earlyFireInterval != interval.toMilliseconds) {
-      // earlyFireInterval of the two query config is not equal and not the default
-      throw new RuntimeException(
-        "Currently not support different earlyFireInterval configs in one job")
-    }
-    earlyFireInterval = interval.toMilliseconds
-    this
-  }
-
-  def getEarlyFireInterval: Long = earlyFireInterval
-
-  /**
-    * Specifies the late firing interval in milli second, early fire is the emit strategy
-    * after watermark advanced to end of window.
-    */
-  def withLateFireInterval(interval: Time): TableConfig = {
-    if (this.lateFireInterval != DEFAULT_FIRE_INTERVAL
-      && this.lateFireInterval != interval.toMilliseconds) {
-      // lateFireInterval of the two query config is not equal and not the default
-      throw new RuntimeException(
-        "Currently not support different lateFireInterval configs in one job")
-    }
-    lateFireInterval = interval.toMilliseconds
-    this
-  }
-
-  def getLateFireInterval: Long = lateFireInterval
-
-  /**
-    * Specifies a minimum and a maximum time interval for how long idle state, i.e., state which
-    * was not updated, will be retained.
-    * State will never be cleared until it was idle for less than the minimum time and will never
-    * be kept if it was idle for more than the maximum time.
-    *
-    * <p>When new data arrives for previously cleaned-up state, the new data will be handled as
-    * if it was the first data. This can result in previous results being overwritten.
-    *
-    * <p>Set to 0 (zero) to never clean-up the state.
-    *
-    * <p>NOTE: Cleaning up state requires additional bookkeeping which becomes less expensive for
-    * larger differences of minTime and maxTime. The difference between minTime and maxTime must be
-    * at least 5 minutes.
-    *
-    * @param minTime The minimum time interval for which idle state is retained. Set to 0 (zero) to
-    * never clean-up the state.
-    * @param maxTime The maximum time interval for which idle state is retained. Must be at least
-    * 5 minutes greater than minTime. Set to 0 (zero) to never clean-up the state.
-    */
-  def setIdleStateRetentionTime(minTime: Time, maxTime: Time): Unit = {
-    if (maxTime.toMilliseconds - minTime.toMilliseconds < 300000 &&
-      !(maxTime.toMilliseconds == 0) && minTime.toMilliseconds == 0) {
-      throw new IllegalArgumentException(
-        "Difference between minTime: " + minTime.toString + " and maxTime: " + maxTime.toString +
-          "should be at least 5 minutes.")
-    }
-    minIdleStateRetentionTime = minTime.toMilliseconds
-    maxIdleStateRetentionTime = maxTime.toMilliseconds
-  }
-}
-
-object TableConfig {
-  def DEFAULT = new TableConfig()
-}
-
-object OperatorType extends Enumeration {
-  type OperatorType = Value
-  val NestedLoopJoin, ShuffleHashJoin, BroadcastHashJoin, SortMergeJoin, HashAgg, SortAgg = Value
-}
-
-object AggPhaseEnforcer extends Enumeration {
-  type AggPhaseEnforcer = Value
-  val AUTO, ONE_PHASE, TWO_PHASE = Value
-}
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
index 8137723..e0a8fcc 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/calcite/CalciteConfig.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.calcite
 
+import org.apache.flink.table.api.PlannerConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, StreamOptimizeContext}
 import org.apache.flink.util.Preconditions
 
@@ -146,7 +147,7 @@ class CalciteConfigBuilder {
 /**
   * Calcite configuration for defining a custom Calcite configuration for Table and SQL API.
   */
-trait CalciteConfig {
+trait CalciteConfig extends PlannerConfig {
 
   /**
     * Returns a custom batch table optimize program
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala
index 712f63c..999a034 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/ExpressionReducer.scala
@@ -25,7 +25,7 @@ import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.FunctionCodeGenerator.generateFunction
 import org.apache.flink.table.dataformat.BinaryStringUtil.safeToString
-import org.apache.flink.table.dataformat.{BinaryString, BinaryStringUtil, Decimal, GenericRow}
+import org.apache.flink.table.dataformat.{BinaryString, Decimal, GenericRow}
 import org.apache.flink.table.functions.{FunctionContext, UserDefinedFunction}
 import org.apache.flink.table.types.logical.RowType
 
@@ -100,7 +100,11 @@ class ExpressionReducer(
       case _ => throw new TableException("RichMapFunction[GenericRow, GenericRow] required here")
     }
 
-    val parameters = if (config.getConf != null) config.getConf else new Configuration()
+    val parameters = if (config.getConfiguration != null) {
+      config.getConfiguration
+    } else {
+      new Configuration()
+    }
     val reduced = try {
       richMapFunction.open(parameters)
       // execute
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
index 8e8cbd8..e5ce158 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/codegen/NestedLoopJoinCodeGenerator.scala
@@ -70,7 +70,7 @@ class NestedLoopJoinCodeGenerator(
     val isFirstRow = newName("isFirstRow")
     val isBinaryRow = newName("isBinaryRow")
 
-    val externalBufferMemorySize = config.getConf.getInteger(
+    val externalBufferMemorySize = config.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM) * NodeResourceConfig.SIZE_IN_MB
 
     if (singleRowJoin) {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnNullCount.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnNullCount.scala
index c63189f..2e98698 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnNullCount.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdColumnNullCount.scala
@@ -159,7 +159,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.getConf.getInteger(
+        val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
           SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT)
         val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate)
         val conjunctions = RelOptUtil.conjunctions(cnf)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistinctRowCount.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistinctRowCount.scala
index e5dbcfe..60b3382 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistinctRowCount.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistinctRowCount.scala
@@ -204,7 +204,7 @@ class FlinkRelMdDistinctRowCount private extends MetadataHandler[BuiltInMetadata
     } else {
       val rexBuilder = rel.getCluster.getRexBuilder
       val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
-      val maxCnfNodeCount = tableConfig.getConf.getInteger(
+      val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
         SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT)
       val cnf = FlinkRexUtil.toCnf(rexBuilder, maxCnfNodeCount, predicate)
       val conjunctions = RelOptUtil.conjunctions(cnf)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistribution.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistribution.scala
index a637ae6..c4c1e1b8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistribution.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdDistribution.scala
@@ -72,7 +72,7 @@ class FlinkRelMdDistribution private extends MetadataHandler[FlinkDistribution]
 
   def flinkDistribution(sort: Sort, mq: RelMetadataQuery): FlinkRelDistribution = {
     val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(sort)
-    val enableRangeSort = tableConfig.getConf.getBoolean(
+    val enableRangeSort = tableConfig.getConfiguration.getBoolean(
       BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED)
     if ((sort.getCollation.getFieldCollations.nonEmpty &&
       sort.fetch == null && sort.offset == null) && enableRangeSort) {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdRowCount.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdRowCount.scala
index 6b57772..24f4183 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdRowCount.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/FlinkRelMdRowCount.scala
@@ -177,7 +177,8 @@ class FlinkRelMdRowCount private extends MetadataHandler[BuiltInMetadata.RowCoun
       val inputRowCnt = mq.getRowCount(input)
       val config = rel.getCluster.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
       val parallelism = (inputRowCnt /
-          config.getConf.getLong(FlinkRelMdRowCount.SQL_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt
+          config.getConfiguration.getLong(
+            FlinkRelMdRowCount.SQL_OPTIMIZER_ROWS_PER_LOCALAGG) + 1).toInt
       if (parallelism == 1) {
         ndvOfGroupKeysOnGlobalAgg
       } else if (grouping.isEmpty) {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/SelectivityEstimator.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/SelectivityEstimator.scala
index d1d04ef..715189f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/SelectivityEstimator.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/metadata/SelectivityEstimator.scala
@@ -62,7 +62,7 @@ class SelectivityEstimator(rel: RelNode, mq: FlinkRelMetadataQuery)
 
   private val rexBuilder = rel.getCluster.getRexBuilder
   private val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(rel)
-  private val maxCnfNodeCount = tableConfig.getConf.getInteger(
+  private val maxCnfNodeCount = tableConfig.getConfiguration.getInteger(
     SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT)
 
   // these default values is referred to RelMdUtil#guessSelectivity
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
index 54308ca..45a6266 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/common/CommonLookupJoin.scala
@@ -17,15 +17,15 @@
  */
 package org.apache.flink.table.plan.nodes.common
 
-import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.dag.Transformation
 import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo, TypeExtractor}
 import org.apache.flink.streaming.api.datastream.AsyncDataStream.OutputMode
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
 import org.apache.flink.streaming.api.operators.ProcessOperator
 import org.apache.flink.streaming.api.operators.async.AsyncWaitOperator
 import org.apache.flink.streaming.api.transformations.OneInputTransformation
-import org.apache.flink.table.api.{TableConfig, ExecutionConfigOptions, TableException, TableSchema}
+import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig, TableException, TableSchema}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.codegen.LookupJoinCodeGenerator._
 import org.apache.flink.table.codegen.{CodeGeneratorContext, LookupJoinCodeGenerator}
@@ -44,7 +44,10 @@ import org.apache.flink.table.types.PlannerTypeUtils.isInteroperable
 import org.apache.flink.table.types.logical.{LogicalType, RowType, TypeInformationAnyType}
 import org.apache.flink.table.types.utils.TypeConversions.fromDataTypeToLegacyInfo
 import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 import org.apache.flink.types.Row
+
+import com.google.common.primitives.Primitives
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.{RelDataType, RelDataTypeField}
 import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
@@ -55,15 +58,12 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable
 import org.apache.calcite.sql.validate.SqlValidatorUtil
 import org.apache.calcite.tools.RelBuilder
 import org.apache.calcite.util.mapping.IntPair
-import com.google.common.primitives.Primitives
+
 import java.util.Collections
 import java.util.concurrent.CompletableFuture
 
-import org.apache.flink.api.dag.Transformation
-
 import scala.collection.JavaConverters._
 import scala.collection.mutable
-import scala.concurrent.duration.Duration
 
 /**
   * Common abstract RelNode for temporal table join which shares most methods.
@@ -186,9 +186,9 @@ abstract class CommonLookupJoin(
     val leftOuterJoin = joinType == JoinRelType.LEFT
 
     val operator = if (lookupableTableSource.isAsyncEnabled) {
-      val asyncBufferCapacity= config.getConf
+      val asyncBufferCapacity= config.getConfiguration
         .getInteger(ExecutionConfigOptions.SQL_EXEC_LOOKUP_ASYNC_BUFFER_CAPACITY)
-      val asyncTimeout = config.getMillisecondFromConfigDuration(
+      val asyncTimeout = getMillisecondFromConfigDuration(config,
         ExecutionConfigOptions.SQL_EXEC_LOOKUP_ASYNC_TIMEOUT)
 
       val asyncLookupFunction = lookupableTableSource
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala
index 3c890f7..7befb58 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/logical/FlinkLogicalSort.scala
@@ -18,10 +18,10 @@
 
 package org.apache.flink.table.plan.nodes.logical
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.nodes.FlinkConventions
-import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule
+import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
 import org.apache.flink.table.plan.util.SortUtil
 
 import org.apache.calcite.plan._
@@ -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.asInstanceOf[FlinkContext].getTableConfig
-    val enableRangeSort = config.getConf.getBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED)
-    val limitValue = config.getConf.getInteger(ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT)
+    val enableRangeSort = config.getConfiguration.getBoolean(SQL_EXEC_SORT_RANGE_ENABLED)
+    val limitValue = config.getConfiguration.getInteger(SQL_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-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala
index 83db844..4a04716 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecGroupAggregateBase.scala
@@ -18,9 +18,11 @@
 
 package org.apache.flink.table.plan.nodes.physical.batch
 
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions, TableException}
+import org.apache.flink.table.api.TableException
 import org.apache.flink.table.functions.UserDefinedFunction
 import org.apache.flink.table.plan.util.{FlinkRelOptUtil, RelExplainUtil}
+import org.apache.flink.table.util.AggregatePhaseStrategy
+import org.apache.flink.table.util.TableConfigUtils.getAggPhaseStrategy
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
 import org.apache.calcite.rel.`type`.RelDataType
@@ -86,9 +88,7 @@ abstract class BatchExecGroupAggregateBase(
 
   protected def isEnforceTwoStageAgg: Boolean = {
     val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-    val aggConfig = tableConfig.getConf.getString(
-      OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY)
-    AggPhaseEnforcer.TWO_PHASE.toString.equalsIgnoreCase(aggConfig)
+    getAggPhaseStrategy(tableConfig) == AggregatePhaseStrategy.TWO_PHASE
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala
index b9315e3..e7988ce 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregate.scala
@@ -113,7 +113,7 @@ class BatchExecHashAggregate(
         } else {
           // If partialKey is enabled, try to use partial key to satisfy the required distribution
           val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-          val partialKeyEnabled = tableConfig.getConf.getBoolean(
+          val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
             BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
           partialKeyEnabled && groupKeysList.containsAll(shuffleKeys)
         }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
index f4b333d..7734ada 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashAggregateBase.scala
@@ -135,7 +135,7 @@ abstract class BatchExecHashAggregateBase(
       AggWithoutKeysCodeGenerator.genWithoutKeys(
         ctx, relBuilder, aggInfos, inputType, outputType, isMerge, isFinal, "NoGrouping")
     } else {
-      val reservedManagedMem = config.getConf.getInteger(
+      val reservedManagedMem = config.getConfiguration.getInteger(
         ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM) * NodeResourceConfig.SIZE_IN_MB
       new HashAggCodeGenerator(
         ctx, relBuilder, aggInfos, inputType, outputType, grouping, auxGrouping, isMerge, isFinal
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
index 3b80921..0e64fcf 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashJoin.scala
@@ -201,7 +201,7 @@ class BatchExecHashJoin(
     val rType = rInput.getOutputType.asInstanceOf[BaseRowTypeInfo].toRowType
 
     val keyType = RowType.of(leftKeys.map(lType.getChildren().get(_)): _*)
-    val managedMemorySize = config.getConf.getInteger(
+    val managedMemorySize = config.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM) * NodeResourceConfig.SIZE_IN_MB
     val condFunc = JoinUtil.generateConditionFunction(
       config, cluster.getRexBuilder, getJoinInfo, lType, rType)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
index f74236a..341fe0b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecHashWindowAggregateBase.scala
@@ -130,12 +130,12 @@ abstract class BatchExecHashWindowAggregateBase(
     val aggInfos = transformToBatchAggregateInfoList(
       aggCallToAggFunction.map(_._1), aggInputRowType)
 
-    val groupBufferLimitSize = config.getConf.getInteger(
+    val groupBufferLimitSize = config.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT)
 
     val (windowSize: Long, slideSize: Long) = WindowCodeGenerator.getWindowDef(window)
 
-    val reservedManagedMem = config.getConf.getInteger(
+    val reservedManagedMem = config.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM) * NodeResourceConfig.SIZE_IN_MB
 
     val generatedOperator = new HashWindowCodeGenerator(
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
index afa01f8..f4dee1c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecOverAggregate.scala
@@ -274,7 +274,7 @@ class BatchExecOverAggregate(
           val isAllFieldsFromInput = requiredDistribution.getKeys.forall(_ < inputFieldCnt)
           if (isAllFieldsFromInput) {
             val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-            if (tableConfig.getConf.getBoolean(
+            if (tableConfig.getConfiguration.getBoolean(
               BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)) {
               ImmutableIntList.of(grouping: _*).containsAll(requiredDistribution.getKeys)
             } else {
@@ -409,7 +409,7 @@ class BatchExecOverAggregate(
     } else {
       val windowFrames = createOverWindowFrames(config)
       new BufferDataOverWindowOperator(
-        config.getConf.getInteger(
+        config.getConfiguration.getInteger(
           ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM) * NodeResourceConfig.SIZE_IN_MB,
         windowFrames,
         genComparator,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala
index 348d569..76b4c99 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecRank.scala
@@ -143,7 +143,7 @@ class BatchExecRank(
         } else {
           // If partialKey is enabled, try to use partial key to satisfy the required distribution
           val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-          val partialKeyEnabled = tableConfig.getConf.getBoolean(
+          val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
             BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
           partialKeyEnabled && partitionKeyList.containsAll(shuffleKeys)
         }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala
index 1ca29f2..19aa60e 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSink.scala
@@ -99,7 +99,7 @@ class BatchExecSink[T](
         val sinkTransformation = dsSink.getTransformation
 
         val configSinkParallelism = NodeResourceConfig.getSinkParallelism(
-          planner.getTableConfig.getConf)
+          planner.getTableConfig.getConfiguration)
 
         val maxSinkParallelism = sinkTransformation.getMaxParallelism
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
index f04bce0..00c1fb6 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSort.scala
@@ -116,7 +116,7 @@ class BatchExecSort(
     val keyTypes = keys.map(inputType.getTypeAt)
     val codeGen = new SortCodeGenerator(conf, keys, keyTypes, orders, nullsIsLast)
 
-    val reservedMemorySize = conf.getConf.getInteger(
+    val reservedMemorySize = conf.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM) * NodeResourceConfig.SIZE_IN_MB
 
     val operator = new SortOperator(
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala
index ea6ad20..238358a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortAggregate.scala
@@ -114,7 +114,7 @@ class BatchExecSortAggregate(
         } else {
           // If partialKey is enabled, try to use partial key to satisfy the required distribution
           val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(this)
-          val partialKeyEnabled = tableConfig.getConf.getBoolean(
+          val partialKeyEnabled = tableConfig.getConfiguration.getBoolean(
             BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
           partialKeyEnabled && groupKeysList.containsAll(shuffleKeys)
         }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
index ca85e07..1d9ba48 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortMergeJoin.scala
@@ -219,10 +219,10 @@ class BatchExecSortMergeJoin(
       leftType,
       rightType)
 
-    val externalBufferMemory = config.getConf.getInteger(
+    val externalBufferMemory = config.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM) * NodeResourceConfig.SIZE_IN_MB
 
-    val sortMemory = config.getConf.getInteger(
+    val sortMemory = config.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM) * NodeResourceConfig.SIZE_IN_MB
 
     def newSortGen(originalKeys: Array[Int], t: RowType): SortCodeGenerator = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
index c73d941..ab8be01 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/batch/BatchExecSortWindowAggregateBase.scala
@@ -118,7 +118,7 @@ abstract class BatchExecSortWindowAggregateBase(
     val aggInfos = transformToBatchAggregateInfoList(
       aggCallToAggFunction.map(_._1), aggInputRowType)
 
-    val groupBufferLimitSize = planner.getTableConfig.getConf.getInteger(
+    val groupBufferLimitSize = planner.getTableConfig.getConfiguration.getInteger(
       ExecutionConfigOptions.SQL_EXEC_WINDOW_AGG_BUFFER_SIZE_LIMIT)
 
     val (windowSize: Long, slideSize: Long) = WindowCodeGenerator.getWindowDef(window)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
index a6d1087..581a9a4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecDeduplicate.scala
@@ -113,7 +113,7 @@ class StreamExecDeduplicate(
     val rowTypeInfo = inputTransform.getOutputType.asInstanceOf[BaseRowTypeInfo]
     val generateRetraction = StreamExecRetractionRules.isAccRetract(this)
     val tableConfig = planner.getTableConfig
-    val isMiniBatchEnabled = tableConfig.getConf.getBoolean(
+    val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
     val operator = if (isMiniBatchEnabled) {
       val exeConfig = planner.getExecEnv.getConfig
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
index a5260b8..a9ee7e8 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGlobalGroupAggregate.scala
@@ -162,7 +162,7 @@ class StreamExecGlobalGroupAggregate(
     val recordEqualiser = new EqualiserCodeGenerator(globalAggValueTypes)
       .generateRecordEqualiser("GroupAggValueEqualiser")
 
-    val operator = if (tableConfig.getConf.getBoolean(
+    val operator = if (tableConfig.getConfiguration.getBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)) {
       val aggFunction = new MiniBatchGlobalGroupAggFunction(
         localAggsHandler,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
index 34a3bb7..a9cd9f3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupAggregate.scala
@@ -167,7 +167,7 @@ class StreamExecGroupAggregate(
       .generateRecordEqualiser("GroupAggValueEqualiser")
     val inputCountIndex = aggInfoList.getIndexOfCountStar
 
-    val isMiniBatchEnabled = tableConfig.getConf.getBoolean(
+    val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
 
     val operator = if (isMiniBatchEnabled) {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
index d9de5b1..41ed54a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecGroupWindowAggregate.scala
@@ -47,7 +47,6 @@ import org.apache.calcite.tools.RelBuilder
 
 import java.time.Duration
 import java.util
-import java.util.Calendar
 
 import scala.collection.JavaConversions._
 
@@ -168,9 +167,6 @@ class StreamExecGroupWindowAggregate(
           "excessive state size. You may specify a retention time of 0 to not clean up the state.")
     }
 
-    // validation
-    emitStrategy.checkValidation()
-
     val aggString = RelExplainUtil.streamWindowAggregationToString(
       inputRowType,
       grouping,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
index 47d51ca..abd2e6d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecLimit.scala
@@ -127,7 +127,7 @@ class StreamExecLimit(
       tableConfig, "AlwaysEqualsComparator", Array(), Array(), Array(), Array())
 
     val processFunction = if (generateRetraction) {
-      val cacheSize = tableConfig.getConf.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
+      val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
       new AppendOnlyTopNFunction(
         minIdleStateRetentionTime,
         maxIdleStateRetentionTime,
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala
index 90ff177..7b404d0 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecRank.scala
@@ -154,7 +154,7 @@ class StreamExecRank(
     val sortKeyComparator = ComparatorCodeGenerator.gen(tableConfig, "StreamExecSortComparator",
       sortFields.indices.toArray, sortKeyType.getLogicalTypes, sortDirections, nullsIsLast)
     val generateRetraction = StreamExecRetractionRules.isAccRetract(this)
-    val cacheSize = tableConfig.getConf.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
+    val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
     val minIdleStateRetentionTime = tableConfig.getMinIdleStateRetentionTime
     val maxIdleStateRetentionTime = tableConfig.getMaxIdleStateRetentionTime
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala
index ef17190..768b028 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSink.scala
@@ -125,7 +125,7 @@ class StreamExecSink[T](
             s"However, ${sink.getClass.getCanonicalName} doesn't implement this method.")
         }
         val configSinkParallelism = NodeResourceConfig.getSinkParallelism(
-          planner.getTableConfig.getConf)
+          planner.getTableConfig.getConfiguration)
 
         val maxSinkParallelism = dsSink.getTransformation.getMaxParallelism
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala
index 4cad0e8..e22e5b3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSort.scala
@@ -114,7 +114,7 @@ class StreamExecSort(
   override protected def translateToPlanInternal(
       planner: StreamPlanner): Transformation[BaseRow] = {
     val config = planner.getTableConfig
-    if (!config.getConf.getBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED)) {
+    if (!config.getConfiguration.getBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED)) {
       throw new TableException("Sort on a non-time-attribute field is not supported.")
     }
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala
index 1e003a6..6b5b7d5 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecSortLimit.scala
@@ -151,7 +151,7 @@ class StreamExecSortLimit(
       sortDirections,
       nullsIsLast)
     val generateRetraction = StreamExecRetractionRules.isAccRetract(this)
-    val cacheSize = tableConfig.getConf.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
+    val cacheSize = tableConfig.getConfiguration.getLong(StreamExecRank.SQL_EXEC_TOPN_CACHE_SIZE)
     val minIdleStateRetentionTime = tableConfig.getMinIdleStateRetentionTime
     val maxIdleStateRetentionTime = tableConfig.getMaxIdleStateRetentionTime
 
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
index 8dc39b7..1abc009 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/nodes/physical/stream/StreamExecWatermarkAssigner.scala
@@ -29,6 +29,7 @@ import org.apache.flink.table.plan.nodes.exec.{ExecNode, StreamExecNode}
 import org.apache.flink.table.planner.StreamPlanner
 import org.apache.flink.table.runtime.watermarkassigner.{MiniBatchAssignerOperator, MiniBatchedWatermarkAssignerOperator, WatermarkAssignerOperator}
 import org.apache.flink.table.typeutils.BaseRowTypeInfo
+import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 import org.apache.flink.util.Preconditions
 
 import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
@@ -82,7 +83,7 @@ class StreamExecWatermarkAssigner(
       "None"
     } else if (miniBatchInterval.mode == MiniBatchMode.ProcTime) {
       val tableConfig = cluster.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-      val miniBatchLatency = tableConfig.getMillisecondFromConfigDuration(
+      val miniBatchLatency = getMillisecondFromConfigDuration(tableConfig,
         ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY)
       Preconditions.checkArgument(miniBatchLatency > 0,
         "MiniBatch latency must be greater that 0 ms.", null)
@@ -115,7 +116,7 @@ class StreamExecWatermarkAssigner(
     val config = planner.getTableConfig
     val inferredInterval = getTraitSet.getTrait(
       MiniBatchIntervalTraitDef.INSTANCE).getMiniBatchInterval
-    val idleTimeout = config.getMillisecondFromConfigDuration(
+    val idleTimeout = getMillisecondFromConfigDuration(config,
       ExecutionConfigOptions.SQL_EXEC_SOURCE_IDLE_TIMEOUT)
 
     val (operator, opName) = if (inferredInterval.mode == MiniBatchMode.None ||
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
index 0fd68c7..0c5fa9c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/BatchCommonSubGraphBasedOptimizer.scala
@@ -24,6 +24,7 @@ import org.apache.flink.table.plan.nodes.physical.batch.BatchExecSink
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram}
 import org.apache.flink.table.plan.schema.IntermediateRelTable
 import org.apache.flink.table.planner.BatchPlanner
+import org.apache.flink.table.util.TableConfigUtils
 import org.apache.flink.util.Preconditions
 
 import org.apache.calcite.rel.RelNode
@@ -72,8 +73,8 @@ class BatchCommonSubGraphBasedOptimizer(planner: BatchPlanner)
     */
   private def optimizeTree(relNode: RelNode): RelNode = {
     val config = planner.getTableConfig
-    val programs = config.getCalciteConfig.getBatchProgram
-      .getOrElse(FlinkBatchProgram.buildProgram(config.getConf))
+    val programs = TableConfigUtils.getCalciteConfig(config).getBatchProgram
+      .getOrElse(FlinkBatchProgram.buildProgram(config.getConfiguration))
     Preconditions.checkNotNull(programs)
 
     programs.optimize(relNode, new BatchOptimizeContext {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala
index 288d85f..b397070 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/RelNodeBlock.scala
@@ -257,7 +257,7 @@ class RelNodeBlockPlanBuilder private(config: TableConfig) {
   private val node2Wrapper = new util.IdentityHashMap[RelNode, RelNodeWrapper]()
   private val node2Block = new util.IdentityHashMap[RelNode, RelNodeBlock]()
 
-  private val isUnionAllAsBreakPointDisabled = config.getConf.getBoolean(
+  private val isUnionAllAsBreakPointDisabled = config.getConfiguration.getBoolean(
     RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED)
 
   /**
@@ -429,7 +429,7 @@ object RelNodeBlockPlanBuilder {
     * @return RelNode dag which reuse common subPlan in each tree
     */
   private def reuseRelNodes(relNodes: Seq[RelNode], tableConfig: TableConfig): Seq[RelNode] = {
-    val findOpBlockWithDigest = tableConfig.getConf.getBoolean(
+    val findOpBlockWithDigest = tableConfig.getConfiguration.getBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED)
     if (!findOpBlockWithDigest) {
       return relNodes
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
index 9857094..c0196e9 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/optimize/StreamCommonSubGraphBasedOptimizer.scala
@@ -30,6 +30,8 @@ import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.plan.util.FlinkRelOptUtil
 import org.apache.flink.table.planner.StreamPlanner
 import org.apache.flink.table.sinks.{DataStreamTableSink, RetractStreamTableSink}
+import org.apache.flink.table.util.TableConfigUtils
+import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 import org.apache.flink.util.Preconditions
 
 import org.apache.calcite.rel.RelNode
@@ -63,9 +65,9 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner)
           o.getTraitSet.getTrait(UpdateAsRetractionTraitDef.INSTANCE).sendsUpdatesAsRetractions
       }
       sinkBlock.setUpdateAsRetraction(retractionFromRoot)
-      val miniBatchInterval: MiniBatchInterval = if (config.getConf.getBoolean(
+      val miniBatchInterval: MiniBatchInterval = if (config.getConfiguration.getBoolean(
         ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)) {
-        val miniBatchLatency = config.getMillisecondFromConfigDuration(
+        val miniBatchLatency = getMillisecondFromConfigDuration(config,
           ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY)
         Preconditions.checkArgument(miniBatchLatency > 0,
           "MiniBatch Latency must be greater than 0 ms.", null)
@@ -154,8 +156,9 @@ class StreamCommonSubGraphBasedOptimizer(planner: StreamPlanner)
       isSinkBlock: Boolean): RelNode = {
 
     val config = planner.getTableConfig
-    val programs = config.getCalciteConfig.getStreamProgram
-      .getOrElse(FlinkStreamProgram.buildProgram(config.getConf))
+    val calciteConfig = TableConfigUtils.getCalciteConfig(config)
+    val programs = calciteConfig.getStreamProgram
+      .getOrElse(FlinkStreamProgram.buildProgram(config.getConfiguration))
     Preconditions.checkNotNull(programs)
 
     programs.optimize(relNode, new StreamOptimizeContext() {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala
index eabeaf4..548efa0 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/reuse/SubplanReuser.scala
@@ -55,11 +55,11 @@ object SubplanReuser {
     * Finds duplicated sub-plans and return the reused plan.
     */
   def reuseDuplicatedSubplan(rels: Seq[RelNode], tableConfig: TableConfig): Seq[RelNode] = {
-    if (!tableConfig.getConf.getBoolean(
+    if (!tableConfig.getConfiguration.getBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED)) {
       return rels
     }
-    val tableSourceReuseEnabled = tableConfig.getConf.getBoolean(
+    val tableSourceReuseEnabled = tableConfig.getConfiguration.getBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED)
     val context = new SubplanReuseContext(tableSourceReuseEnabled, rels: _*)
     val reuseShuttle = new SubplanReuseShuttle(context)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRule.scala
index 171844a..6bfdab3 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/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.getConf.getLong(
+    val minNullCount = conf.getConfiguration.getLong(
       JoinDeriveNullFilterRule.SQL_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD)
 
     def createIsNotNullFilter(input: RelNode, keys: ImmutableIntList): RelNode = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
index ada01e3..a879687 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRule.scala
@@ -46,7 +46,7 @@ class PushFilterIntoTableSourceScanRule extends RelOptRule(
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val config = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    if (!config.getConf.getBoolean(
+    if (!config.getConfiguration.getBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_PREDICATE_PUSHDOWN_ENABLED)) {
       return false
     }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala
index 922f60b..3e86e44 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRule.scala
@@ -114,9 +114,9 @@ class SplitAggregateRule extends RelOptRule(
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
     val agg: FlinkLogicalAggregate = call.rel(0)
 
-    val isMiniBatchEnabled = tableConfig.getConf.getBoolean(
+    val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
-    val splitSkewDistinctAggEnabled = tableConfig.getConf.getBoolean(
+    val splitSkewDistinctAggEnabled = tableConfig.getConfiguration.getBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED)
     val isAllAggSplittable = doAllAggSupportSplit(agg.getAggCallList)
 
@@ -144,7 +144,7 @@ class SplitAggregateRule extends RelOptRule(
     }.distinct.diff(aggGroupSet).sorted.toArray
 
     val hashFieldsMap: util.Map[Int, Int] = new util.HashMap()
-    val buckets = tableConfig.getConf.getInteger(
+    val buckets = tableConfig.getConfiguration.getInteger(
       OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM)
 
     if (hashFieldIndexes.nonEmpty) {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala
index 866270f..430435b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecAggRuleBase.scala
@@ -18,7 +18,7 @@
 package org.apache.flink.table.plan.rules.physical.batch
 
 import org.apache.flink.table.JArrayList
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions, TableConfig, TableException}
+import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.calcite.FlinkTypeFactory
 import org.apache.flink.table.dataformat.BinaryRow
 import org.apache.flink.table.functions.aggfunctions.DeclarativeAggregateFunction
@@ -27,6 +27,8 @@ import org.apache.flink.table.functions.{AggregateFunction, UserDefinedFunction}
 import org.apache.flink.table.plan.util.{AggregateUtil, FlinkRelOptUtil}
 import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
 import org.apache.flink.table.types.logical.LogicalType
+import org.apache.flink.table.util.AggregatePhaseStrategy
+import org.apache.flink.table.util.TableConfigUtils.getAggPhaseStrategy
 
 import org.apache.calcite.rel.`type`.RelDataType
 import org.apache.calcite.rel.core.Aggregate
@@ -102,8 +104,8 @@ trait BatchExecAggRuleBase {
   protected def isTwoPhaseAggWorkable(
       aggFunctions: Array[UserDefinedFunction],
       tableConfig: TableConfig): Boolean = {
-    getAggEnforceStrategy(tableConfig) match {
-      case AggPhaseEnforcer.ONE_PHASE => false
+    getAggPhaseStrategy(tableConfig) match {
+      case AggregatePhaseStrategy.ONE_PHASE => false
       case _ => doAllSupportMerge(aggFunctions)
     }
   }
@@ -112,10 +114,10 @@ trait BatchExecAggRuleBase {
       agg: Aggregate,
       aggFunctions: Array[UserDefinedFunction],
       tableConfig: TableConfig): Boolean = {
-    getAggEnforceStrategy(tableConfig) match {
-      case AggPhaseEnforcer.ONE_PHASE => true
-      case AggPhaseEnforcer.TWO_PHASE => !doAllSupportMerge(aggFunctions)
-      case AggPhaseEnforcer.AUTO =>
+    getAggPhaseStrategy(tableConfig) match {
+      case AggregatePhaseStrategy.ONE_PHASE => true
+      case AggregatePhaseStrategy.TWO_PHASE => !doAllSupportMerge(aggFunctions)
+      case AggregatePhaseStrategy.AUTO =>
         if (!doAllSupportMerge(aggFunctions)) {
           true
         } else {
@@ -138,19 +140,11 @@ trait BatchExecAggRuleBase {
   }
 
   protected def isEnforceOnePhaseAgg(tableConfig: TableConfig): Boolean = {
-    getAggEnforceStrategy(tableConfig) == AggPhaseEnforcer.ONE_PHASE
+    getAggPhaseStrategy(tableConfig) == AggregatePhaseStrategy.ONE_PHASE
   }
 
   protected def isEnforceTwoPhaseAgg(tableConfig: TableConfig): Boolean = {
-    getAggEnforceStrategy(tableConfig) == AggPhaseEnforcer.TWO_PHASE
-  }
-
-  protected def getAggEnforceStrategy(tableConfig: TableConfig): AggPhaseEnforcer.Value = {
-    val aggPrefConfig = tableConfig.getConf.getString(
-      OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY)
-    AggPhaseEnforcer.values.find(_.toString.equalsIgnoreCase(aggPrefConfig))
-      .getOrElse(throw new IllegalArgumentException(
-        "Agg phase enforcer can only set to be: NONE, ONE_PHASE, TWO_PHASE!"))
+    getAggPhaseStrategy(tableConfig) == AggregatePhaseStrategy.TWO_PHASE
   }
 
   protected def isAggBufferFixedLength(agg: Aggregate): Boolean = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
index 85a7bcf..dd2ddfd 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashAggRule.scala
@@ -17,14 +17,15 @@
  */
 package org.apache.flink.table.plan.rules.physical.batch
 
-import org.apache.flink.table.api.{OperatorType, OptimizerConfigOptions}
+import org.apache.flink.table.api.OptimizerConfigOptions
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalAggregate
 import org.apache.flink.table.plan.nodes.physical.batch.{BatchExecHashAggregate, BatchExecLocalHashAggregate}
-import org.apache.flink.table.plan.util.AggregateUtil
+import org.apache.flink.table.plan.util.{AggregateUtil, OperatorType}
 import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.util.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
@@ -63,7 +64,7 @@ class BatchExecHashAggRule
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    if (!tableConfig.isOperatorEnabled(OperatorType.HashAgg)) {
+    if (isOperatorDisabled(tableConfig, OperatorType.HashAgg)) {
       return false
     }
     val agg: FlinkLogicalAggregate = call.rel(0)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala
index a154288..82a280f 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecHashJoinRule.scala
@@ -19,12 +19,14 @@
 package org.apache.flink.table.plan.rules.physical.batch
 
 import org.apache.flink.table.JDouble
-import org.apache.flink.table.api.{OperatorType, OptimizerConfigOptions, TableConfig}
+import org.apache.flink.table.api.{OptimizerConfigOptions, TableConfig}
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.nodes.physical.batch.BatchExecHashJoin
+import org.apache.flink.table.plan.util.OperatorType
+import org.apache.flink.table.util.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
@@ -57,8 +59,9 @@ class BatchExecHashJoinRule
     }
 
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    val isShuffleHashJoinEnabled = tableConfig.isOperatorEnabled(OperatorType.ShuffleHashJoin)
-    val isBroadcastHashJoinEnabled = tableConfig.isOperatorEnabled(OperatorType.BroadcastHashJoin)
+    val isShuffleHashJoinEnabled = !isOperatorDisabled(tableConfig, OperatorType.ShuffleHashJoin)
+    val isBroadcastHashJoinEnabled = !isOperatorDisabled(
+      tableConfig, OperatorType.BroadcastHashJoin)
 
     val leftSize = binaryRowRelNodeSize(join.getLeft)
     val rightSize = binaryRowRelNodeSize(join.getRight)
@@ -141,7 +144,7 @@ class BatchExecHashJoinRule
         toHashTraitByColumns(joinInfo.rightKeys))
 
       // add more possibility to only shuffle by partial joinKeys, now only single one
-      val isShuffleByPartialKeyEnabled = tableConfig.getConf.getBoolean(
+      val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean(
         BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
       if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) {
         joinInfo.pairs().foreach { pair =>
@@ -173,7 +176,7 @@ class BatchExecHashJoinRule
     if (leftSize == null || rightSize == null) {
       return (false, false)
     }
-    val threshold = tableConfig.getConf.getLong(
+    val threshold = tableConfig.getConfiguration.getLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD)
     joinType match {
       case JoinRelType.LEFT => (rightSize <= threshold, false)
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecJoinRuleBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
index 785739a..c832d94 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecJoinRuleBase.scala
@@ -60,7 +60,7 @@ trait BatchExecJoinRuleBase {
       distinctKeys: Seq[Int]): Boolean = {
     val tableConfig = FlinkRelOptUtil.getTableConfigFromContext(buildRel)
     val mq = buildRel.getCluster.getMetadataQuery
-    val ratioConf = tableConfig.getConf.getDouble(
+    val ratioConf = tableConfig.getConfiguration.getDouble(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SEMI_JOIN_BUILD_DISTINCT_NDV_RATIO)
     val inputRows = mq.getRowCount(buildRel)
     val ndvOfGroupKey = mq.getDistinctRowCount(
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecNestedLoopJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecNestedLoopJoinRule.scala
index ebe2459..75fd39b 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecNestedLoopJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecNestedLoopJoinRule.scala
@@ -17,10 +17,11 @@
  */
 package org.apache.flink.table.plan.rules.physical.batch
 
-import org.apache.flink.table.api.OperatorType
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.nodes.physical.batch.BatchExecNestedLoopJoin
+import org.apache.flink.table.plan.util.OperatorType
+import org.apache.flink.table.util.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
@@ -41,7 +42,7 @@ class BatchExecNestedLoopJoinRule
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    tableConfig.isOperatorEnabled(OperatorType.NestedLoopJoin)
+    !isOperatorDisabled(tableConfig, OperatorType.NestedLoopJoin)
   }
 
   override def onMatch(call: RelOptRuleCall): Unit = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
index 1cda69e..40abde4 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortAggRule.scala
@@ -17,14 +17,15 @@
  */
 package org.apache.flink.table.plan.rules.physical.batch
 
-import org.apache.flink.table.api.{OperatorType, OptimizerConfigOptions}
+import org.apache.flink.table.api.OptimizerConfigOptions
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalAggregate
 import org.apache.flink.table.plan.nodes.physical.batch.{BatchExecLocalSortAggregate, BatchExecSortAggregate}
-import org.apache.flink.table.plan.util.AggregateUtil
+import org.apache.flink.table.plan.util.{AggregateUtil, OperatorType}
 import org.apache.flink.table.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType
+import org.apache.flink.table.util.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
@@ -65,7 +66,7 @@ class BatchExecSortAggRule
 
   override def matches(call: RelOptRuleCall): Boolean = {
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    tableConfig.isOperatorEnabled(OperatorType.SortAgg)
+    !isOperatorDisabled(tableConfig, OperatorType.SortAgg)
   }
 
   override def onMatch(call: RelOptRuleCall): Unit = {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
index ebb8e85..85b4a81 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortMergeJoinRule.scala
@@ -21,13 +21,13 @@ package org.apache.flink.table.plan.rules.physical.batch
 import org.apache.flink.annotation.Experimental
 import org.apache.flink.configuration.ConfigOption
 import org.apache.flink.configuration.ConfigOptions.key
-import org.apache.flink.table.api.OperatorType
 import org.apache.flink.table.calcite.FlinkContext
 import org.apache.flink.table.plan.`trait`.FlinkRelDistribution
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin
 import org.apache.flink.table.plan.nodes.physical.batch.BatchExecSortMergeJoin
-import org.apache.flink.table.plan.util.FlinkRelOptUtil
+import org.apache.flink.table.plan.util.{FlinkRelOptUtil, OperatorType}
+import org.apache.flink.table.util.TableConfigUtils.isOperatorDisabled
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall, RelTraitSet}
@@ -54,7 +54,7 @@ class BatchExecSortMergeJoinRule
     val join: Join = call.rel(0)
     val joinInfo = join.analyzeCondition
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    val isSortMergeJoinEnabled = tableConfig.isOperatorEnabled(OperatorType.SortMergeJoin)
+    val isSortMergeJoinEnabled = !isOperatorDisabled(tableConfig, OperatorType.SortMergeJoin)
     !joinInfo.pairs().isEmpty && isSortMergeJoinEnabled
   }
 
@@ -109,7 +109,7 @@ class BatchExecSortMergeJoinRule
     }
 
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    val candidates = if (tableConfig.getConf.getBoolean(
+    val candidates = if (tableConfig.getConfiguration.getBoolean(
       BatchExecSortMergeJoinRule.SQL_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 BatchExecSortMergeJoinRule
     }
 
     // add more possibility to only shuffle by partial joinKeys, now only single one
-    val isShuffleByPartialKeyEnabled = tableConfig.getConf.getBoolean(
+    val isShuffleByPartialKeyEnabled = tableConfig.getConfiguration.getBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED)
     if (isShuffleByPartialKeyEnabled && joinInfo.pairs().length > 1) {
       joinInfo.pairs().foreach { pair =>
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortRule.scala
index b5b098e..719509d 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/batch/BatchExecSortRule.scala
@@ -53,7 +53,8 @@ class BatchExecSortRule extends ConverterRule(
     val sort: FlinkLogicalSort = rel.asInstanceOf[FlinkLogicalSort]
     val input = sort.getInput
     val config = sort.getCluster.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
-    val enableRangeSort = config.getConf.getBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED)
+    val enableRangeSort = config.getConfiguration.getBoolean(
+      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED)
     val distribution = if (enableRangeSort) {
       FlinkRelDistribution.range(sort.getCollation.getFieldCollations)
     } else {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/IncrementalAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/IncrementalAggregateRule.scala
index c86011f..9abc611 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/IncrementalAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/IncrementalAggregateRule.scala
@@ -54,7 +54,7 @@ class IncrementalAggregateRule
     val tableConfig = call.getPlanner.getContext.asInstanceOf[FlinkContext].getTableConfig
 
     // whether incremental aggregate is enabled
-    val incrementalAggEnabled = tableConfig.getConf.getBoolean(
+    val incrementalAggEnabled = tableConfig.getConfiguration.getBoolean(
       IncrementalAggregateRule.SQL_OPTIMIZER_INCREMENTAL_AGG_ENABLED)
 
     partialGlobalAgg.partialFinalType == PartialFinalType.PARTIAL &&
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
index 3428225..3b49fa1 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/MiniBatchIntervalInferRule.scala
@@ -64,7 +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.getConf.getBoolean(
+    val miniBatchEnabled = config.getConfiguration.getBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
 
     val updatedTrait = rel match {
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
index fc1376e..e008744 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/rules/physical/stream/TwoStageOptimizedAggregateRule.scala
@@ -18,7 +18,7 @@
 
 package org.apache.flink.table.plan.rules.physical.stream
 
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions, ExecutionConfigOptions}
+import org.apache.flink.table.api.ExecutionConfigOptions
 import org.apache.flink.table.calcite.{FlinkContext, FlinkTypeFactory}
 import org.apache.flink.table.plan.`trait`.{AccMode, AccModeTrait, FlinkRelDistribution, FlinkRelDistributionTraitDef}
 import org.apache.flink.table.plan.metadata.FlinkRelMetadataQuery
@@ -26,6 +26,8 @@ import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.physical.stream._
 import org.apache.flink.table.plan.rules.physical.FlinkExpandConversionRule._
 import org.apache.flink.table.plan.util.{AggregateInfoList, AggregateUtil}
+import org.apache.flink.table.util.AggregatePhaseStrategy
+import org.apache.flink.table.util.TableConfigUtils.getAggPhaseStrategy
 
 import org.apache.calcite.plan.RelOptRule.{any, operand}
 import org.apache.calcite.plan.{RelOptRule, RelOptRuleCall}
@@ -73,11 +75,9 @@ class TwoStageOptimizedAggregateRule extends RelOptRule(
       needRetraction,
       isStateBackendDataViews = true)
 
-    val isMiniBatchEnabled = tableConfig.getConf.getBoolean(
+    val isMiniBatchEnabled = tableConfig.getConfiguration.getBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED)
-    val isTwoPhaseEnabled = !tableConfig.getConf
-      .getString(OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY)
-      .equalsIgnoreCase(AggPhaseEnforcer.ONE_PHASE.toString)
+    val isTwoPhaseEnabled = getAggPhaseStrategy(tableConfig) != AggregatePhaseStrategy.ONE_PHASE
 
     isMiniBatchEnabled && isTwoPhaseEnabled &&
       AggregateUtil.doAllSupportPartialMerge(aggInfoList.aggInfos) &&
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
index e512a09b..31784aa 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/AggregateUtil.scala
@@ -671,7 +671,7 @@ object AggregateUtil extends Enumeration {
     * Creates a MiniBatch trigger depends on the config.
     */
   def createMiniBatchTrigger(tableConfig: TableConfig): CountBundleTrigger[BaseRow] = {
-    val size = tableConfig.getConf.getLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE)
+    val size = tableConfig.getConfiguration.getLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE)
     if (size <= 0 ) {
       throw new IllegalArgumentException(
         ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE + " must be > 0.")
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelOptUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelOptUtil.scala
index de71cfc..f2fe8bc 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelOptUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/FlinkRelOptUtil.scala
@@ -149,7 +149,7 @@ object FlinkRelOptUtil {
   /** Get max cnf node limit by context of rel */
   def getMaxCnfNodeCount(rel: RelNode): Int = {
     val tableConfig = getTableConfigFromContext(rel)
-    tableConfig.getConf.getInteger(SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT)
+    tableConfig.getConfiguration.getInteger(SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT)
   }
 
   /**
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala
index 1cb1593..794674c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/RankUtil.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.util
 
 import org.apache.flink.table.api.TableConfig
 import org.apache.flink.table.codegen.ExpressionReducer
-import org.apache.flink.table.plan.metadata.SelectivityEstimator
+import org.apache.flink.table.plan.metadata.SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT
 import org.apache.flink.table.plan.nodes.calcite.Rank
 import org.apache.flink.table.runtime.rank.{ConstantRankRange, ConstantRankRangeWithoutEnd, RankRange, VariableRankRange}
 
@@ -71,7 +71,7 @@ object RankUtil {
       config: TableConfig): (Option[RankRange], Option[RexNode]) = {
 
     // Converts the condition to conjunctive normal form (CNF)
-    val cnfNodeCount = config.getConf.getInteger(SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT)
+    val cnfNodeCount = config.getConfiguration.getInteger(SQL_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-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala
index 6c88470..8657e6c 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowEmitStrategy.scala
@@ -17,44 +17,61 @@
  */
 package org.apache.flink.table.plan.util
 
+import org.apache.flink.annotation.Experimental
+import org.apache.flink.configuration.ConfigOption
+import org.apache.flink.configuration.ConfigOptions.key
 import org.apache.flink.table.api.window.TimeWindow
 import org.apache.flink.table.api.{TableConfig, TableException}
 import org.apache.flink.table.plan.logical.{LogicalWindow, SessionGroupWindow}
 import org.apache.flink.table.plan.util.AggregateUtil.isRowtimeAttribute
 import org.apache.flink.table.runtime.window.triggers._
+import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 
+import java.lang.{Boolean, Long}
 import java.time.Duration
 
 class WindowEmitStrategy(
     isEventTime: Boolean,
     isSessionWindow: Boolean,
-    val earlyFireInterval: Long,
-    val lateFireInterval: Long,
+    earlyFireDelay: Long,
+    earlyFireDelayEnabled: Boolean,
+    lateFireDelay: Long,
+    lateFireDelayEnabled: Boolean,
     allowLateness: Long) {
 
+  checkValidation()
+
   def getAllowLateness: Long = allowLateness
 
-  def checkValidation(): Unit = {
-    if (isSessionWindow && (earlyFireInterval >= 0 || lateFireInterval >= 0)) {
+  private def checkValidation(): Unit = {
+    if (isSessionWindow && (earlyFireDelayEnabled || lateFireDelayEnabled)) {
       throw new TableException("Session window doesn't support EMIT strategy currently.")
     }
-    if (isEventTime && lateFireInterval >= 0L && allowLateness <= 0L) {
-      throw new TableException("The 'AFTER WATERMARK' emit strategy requires " +
-        "'sql.exec.state.ttl.ms' config in job config.")
+    if (isEventTime && lateFireDelayEnabled && allowLateness <= 0L) {
+      throw new TableException("The 'AFTER WATERMARK' emit strategy requires set " +
+        "'minIdleStateRetentionTime' in table config.")
+    }
+    if (earlyFireDelayEnabled && (earlyFireDelay == null || earlyFireDelay < 0)) {
+      throw new TableException("Early-fire delay should not be null or negative value when" +
+        "enable early-fire emit strategy.")
+    }
+    if (lateFireDelayEnabled && (lateFireDelay == null || lateFireDelay < 0)) {
+      throw new TableException("Late-fire delay should not be null or negative value when" +
+        "enable late-fire emit strategy.")
     }
   }
 
   def produceUpdates: Boolean = {
     if (isEventTime) {
-      allowLateness > 0 || earlyFireInterval >= 0 || lateFireInterval >= 0
+      allowLateness > 0 || earlyFireDelayEnabled || lateFireDelayEnabled
     } else {
-      earlyFireInterval >= 0
+      earlyFireDelayEnabled
     }
   }
 
   def getTrigger: Trigger[TimeWindow] = {
-    val earlyTrigger = createTriggerFromInterval(earlyFireInterval)
-    val lateTrigger = createTriggerFromInterval(lateFireInterval)
+    val earlyTrigger = createTriggerFromInterval(earlyFireDelayEnabled, earlyFireDelay)
+    val lateTrigger = createTriggerFromInterval(lateFireDelayEnabled, lateFireDelay)
 
     if (isEventTime) {
       val trigger = EventTimeTriggers.afterEndOfWindow[TimeWindow]()
@@ -78,8 +95,8 @@ class WindowEmitStrategy(
 
   override def toString: String = {
     val builder = new StringBuilder
-    val earlyString = intervalToString(earlyFireInterval)
-    val lateString = intervalToString(lateFireInterval)
+    val earlyString = intervalToString(earlyFireDelayEnabled, earlyFireDelay)
+    val lateString = intervalToString(lateFireDelayEnabled, lateFireDelay)
     if (earlyString != null) {
       builder.append("early ").append(earlyString)
     }
@@ -92,23 +109,29 @@ class WindowEmitStrategy(
     builder.toString
   }
 
-  private def createTriggerFromInterval(interval: Long): Option[Trigger[TimeWindow]] = {
-    if (interval > 0) {
-      Some(ProcessingTimeTriggers.every(Duration.ofMillis(interval)))
-    } else if (interval == 0) {
-      Some(ElementTriggers.every())
-    } else {
+  private def createTriggerFromInterval(
+      enableDelayEmit: Boolean,
+      interval: Long): Option[Trigger[TimeWindow]] = {
+    if (!enableDelayEmit) {
       None
+    } else {
+      if (interval > 0) {
+        Some(ProcessingTimeTriggers.every(Duration.ofMillis(interval)))
+      } else {
+        Some(ElementTriggers.every())
+      }
     }
   }
 
-  private def intervalToString(interval: Long): String = {
-    if (interval > 0) {
-      s"delay $interval millisecond"
-    } else if (interval == 0) {
-      "no delay"
-    } else {
+  private def intervalToString(enableDelayEmit: Boolean, interval: Long): String = {
+    if (!enableDelayEmit) {
       null
+    } else {
+      if (interval > 0) {
+        s"delay $interval millisecond"
+      } else {
+        "no delay"
+      }
     }
   }
 }
@@ -128,12 +151,60 @@ object WindowEmitStrategy {
       // use min idle state retention time as allow lateness
       tableConfig.getMinIdleStateRetentionTime
     }
-
+    val enableEarlyFireDelay = tableConfig.getConfiguration.getBoolean(
+      SQL_EXEC_EMIT_EARLY_FIRE_ENABLED)
+    val earlyFireDelay = getMillisecondFromConfigDuration(
+      tableConfig, SQL_EXEC_EMIT_EARLY_FIRE_DELAY)
+    val enableLateFireDelay = tableConfig.getConfiguration.getBoolean(
+      SQL_EXEC_EMIT_LATE_FIRE_ENABLED)
+    val lateFireDelay = getMillisecondFromConfigDuration(
+      tableConfig, SQL_EXEC_EMIT_LATE_FIRE_DELAY)
     new WindowEmitStrategy(
       isEventTime,
       isSessionWindow,
-      tableConfig.getEarlyFireInterval,
-      tableConfig.getLateFireInterval,
+      earlyFireDelay,
+      enableEarlyFireDelay,
+      lateFireDelay,
+      enableLateFireDelay,
       allowLateness)
   }
+
+  // It is a experimental config, will may be removed later.
+  @Experimental
+  val SQL_EXEC_EMIT_EARLY_FIRE_ENABLED: ConfigOption[Boolean] =
+  key("sql.exec.emit.early-fire.enabled")
+      .defaultValue(Boolean.valueOf(false))
+      .withDescription("Specifies whether to enable early-fire emit." +
+          "Early-fire is an emit strategy before watermark advanced to end of window.")
+
+  // It is a experimental config, will may be removed later.
+  @Experimental
+  val SQL_EXEC_EMIT_EARLY_FIRE_DELAY: ConfigOption[String] =
+  key("sql.exec.emit.early-fire.delay")
+      .noDefaultValue
+      .withDescription("The early firing delay in milli second, early fire is " +
+          "the emit strategy before watermark advanced to end of window. " +
+          "< 0 is illegal configuration. " +
+          "0 means no delay (fire on every element). " +
+          "> 0 means the fire interval. ")
+
+  // It is a experimental config, will may be removed later.
+  @Experimental
+  val SQL_EXEC_EMIT_LATE_FIRE_ENABLED: ConfigOption[Boolean] =
+  key("sql.exec.emit.late-fire.enabled")
+      .defaultValue(Boolean.valueOf(false))
+      .withDescription("Specifies whether to enable late-fire emit. " +
+          "Late-fire is an emit strategy after watermark advanced to end of window.")
+
+  // It is a experimental config, will may be removed later.
+  @Experimental
+  val SQL_EXEC_EMIT_LATE_FIRE_DELAY: ConfigOption[String] =
+  key("sql.exec.emit.late-fire.delay")
+      .noDefaultValue
+      .withDescription("The late firing delay in milli second, late fire is " +
+          "the emit strategy after watermark advanced to end of window. " +
+          "< 0 is illegal configuration. " +
+          "0 means no delay (fire on every element). " +
+          "> 0 means the fire interval.")
+
 }
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala
index 91931d6..868217a 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/plan/util/WindowJoinUtil.scala
@@ -170,7 +170,7 @@ object WindowJoinUtil {
 
     // Converts the condition to conjunctive normal form (CNF)
     val cnfCondition = FlinkRexUtil.toCnf(rexBuilder,
-      config.getConf.getInteger(SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT),
+      config.getConfiguration.getInteger(SelectivityEstimator.SQL_OPTIMIZER_CNF_NODES_LIMIT),
       predicate)
 
     // split the condition into time predicates and other predicates
diff --git a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/PlannerBase.scala b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/PlannerBase.scala
index 9a8310d5..3ca99ac 100644
--- a/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/PlannerBase.scala
+++ b/flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/PlannerBase.scala
@@ -257,8 +257,8 @@ abstract class PlannerBase(
     val execEnv = getExecEnv
     if (execEnv != null && execEnv.getConfig != null) {
       val parameters = new Configuration()
-      if (config != null && config.getConf != null) {
-        parameters.addAll(config.getConf)
+      if (config != null && config.getConfiguration != null) {
+        parameters.addAll(config.getConfiguration)
       }
 
       if (execEnv.getConfig.getGlobalJobParameters != null) {
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
index cc8458309..12dd76a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/batch/ExplainTest.scala
@@ -63,7 +63,7 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
   @Test
   def testExplainWithJoin(): Unit = {
     // TODO support other join operators when them are supported
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
     util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extended)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
index 5a58785..83a51fb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/stream/ExplainTest.scala
@@ -109,9 +109,9 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
       "T2", 'id2, 'cnt, 'name, 'goods, 'rowtime.rowtime)
     util.addTableWithWatermark("T3", util.tableEnv.scan("T1"), "rowtime", 0)
     util.addTableWithWatermark("T4", util.tableEnv.scan("T2"), "rowtime", 0)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "3 s")
     val table = util.tableEnv.sqlQuery(
       """
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala
index c67612c..0a8eeda 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/calcite/CalciteConfigBuilderTest.scala
@@ -38,7 +38,7 @@ class CalciteConfigBuilderTest {
     assertTrue(cc.getStreamProgram.isEmpty)
 
     val builder = new CalciteConfigBuilder()
-    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.DEFAULT.getConf)
+    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault.getConfiguration)
     streamPrograms.remove(FlinkStreamProgram.PHYSICAL)
     builder.replaceStreamProgram(streamPrograms)
 
@@ -176,7 +176,7 @@ class CalciteConfigBuilderTest {
     assertTrue(config.getSqlParserConfig.isEmpty)
     assertTrue(config.getSqlToRelConverterConfig.isEmpty)
 
-    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.DEFAULT.getConf)
+    val streamPrograms = FlinkStreamProgram.buildProgram(TableConfig.getDefault.getConfiguration)
     streamPrograms.remove(FlinkStreamProgram.PHYSICAL)
     builder.replaceStreamProgram(streamPrograms)
     val baseConfig1 = builder.build()
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala
index df6a61c..b57e833 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DagOptimizationTest.scala
@@ -135,7 +135,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks1(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
     util.tableEnv.registerTable("table1", table1)
@@ -153,7 +153,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks2(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
 
@@ -174,7 +174,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks3(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
 
@@ -195,7 +195,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks4(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
@@ -221,7 +221,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks5(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     // test with non-deterministic udf
     util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf())
@@ -241,7 +241,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiLevelViews(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
@@ -277,7 +277,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksWithUDTF(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addFunction("split", new TableFunc1)
     val sqlQuery1 =
@@ -316,7 +316,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion1(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table = util.tableEnv.sqlQuery(
@@ -336,9 +336,9 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion2(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
@@ -371,7 +371,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion3(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
@@ -399,7 +399,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion4(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala
index c1a7c4c..c815497 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/DeadlockBreakupTest.scala
@@ -38,9 +38,9 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_SetExchangeAsBatch(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
     val sqlQuery =
       """
@@ -52,11 +52,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_AddExchangeAsBatch_HashJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -68,11 +68,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -84,11 +84,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_SetExchangeAsBatch_SortMergeJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin")
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -104,13 +104,13 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_AddExchangeAsBatch_BuildLeftSemiHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -124,7 +124,7 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testSubplanReuse_SetExchangeAsBatch_OverAgg(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
     val sqlQuery =
       """
@@ -141,11 +141,11 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testReusedNodeIsBarrierNode(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -157,9 +157,9 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testDataStreamReuse_SetExchangeAsBatch(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.a AND t1.b > 10 AND t2.c LIKE 'Test%'"
     util.verifyPlan(sqlQuery)
@@ -167,7 +167,7 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testDataStreamReuse_AddExchangeAsBatch_NestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
     val sqlQuery = "SELECT * FROM t t1, t t2 WHERE t1.a = t2.b"
     util.verifyPlan(sqlQuery)
@@ -175,7 +175,7 @@ class DeadlockBreakupTest extends TableTestBase {
 
   @Test
   def testDataStreamReuse_AddExchangeAsBatch_HashJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
     val sqlQuery = "SELECT * FROM t INTERSECT SELECT * FROM t"
     util.verifyPlan(sqlQuery)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
index bd0ec0f..3969a6d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveCollationTest.scala
@@ -21,7 +21,8 @@ package org.apache.flink.table.plan.batch.sql
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.{ExecutionConfigOptions, Types}
-import org.apache.flink.table.plan.rules.physical.batch.{BatchExecSortMergeJoinRule, BatchExecSortRule}
+import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortMergeJoinRule
+import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
 import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
 import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.StringSplit
 import org.apache.flink.table.util.{TableFunc1, TableTestBase}
@@ -56,13 +57,13 @@ class RemoveCollationTest extends TableTestBase {
       FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
 
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
   }
 
   @Test
   def testRemoveCollation_OverWindowAgg(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,HashAgg")
     val sqlQuery =
       """
@@ -78,7 +79,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Aggregate(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     val sqlQuery =
       """
@@ -90,7 +91,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Aggregate_1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     val sqlQuery =
       """
@@ -102,7 +103,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Sort(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     val sqlQuery =
       """
         |WITH r AS (SELECT a, b, COUNT(c) AS cnt FROM x GROUP BY a, b)
@@ -113,9 +114,9 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Aggregate_3(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     val sqlQuery =
       """
         |WITH r AS (SELECT * FROM x ORDER BY a, b)
@@ -126,7 +127,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -141,7 +142,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -170,7 +171,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_4(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -185,7 +186,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Rank_Singleton(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
     val sqlQuery =
       """
@@ -200,7 +201,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -216,7 +217,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -232,7 +233,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -248,7 +249,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins_MultiJoinKeys2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
 
     val sql =
@@ -264,7 +265,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_MultipleSortMergeJoins3(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     util.addTableSource("tb1",
       Array[TypeInformation[_]](
@@ -340,7 +341,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Correlate1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
@@ -354,7 +355,7 @@ class RemoveCollationTest extends TableTestBase {
 
   @Test
   def testRemoveCollation_Correlate2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
@@ -369,7 +370,7 @@ class RemoveCollationTest extends TableTestBase {
   @Test
   def testRemoveCollation_Correlate3(): Unit = {
     // do not remove shuffle
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
index 9c10057..ad38673 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/RemoveShuffleTest.scala
@@ -43,13 +43,13 @@ class RemoveShuffleTest extends TableTestBase {
       Array("d", "e", "f"),
       FlinkStatistic.builder().tableStats(new TableStats(100L)).build()
     )
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
   }
 
   @Test
   def testRemoveHashShuffle_OverWindowAgg(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -66,7 +66,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_MultiOverWindowAgg(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -83,9 +83,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_OverWindowAgg_PartialKey(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
     // push down HashExchange[c] into HashAgg
     val sqlQuery =
@@ -103,9 +103,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Agg_PartialKey(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
     // push down HashExchange[c] into HashAgg
     val sqlQuery =
@@ -118,10 +118,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashAggregate(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -133,10 +133,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashAggregate_1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -148,10 +148,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashAggregate_2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -163,10 +163,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortAggregate(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -178,10 +178,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortAggregate_1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -193,10 +193,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortAggregate_2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -208,9 +208,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
     val sqlQuery =
       """
@@ -222,9 +222,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin_LOJ(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
     val sqlQuery =
       """
@@ -236,9 +236,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin_ROJ(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
     val sqlQuery =
       """
@@ -250,7 +250,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_SortMergeJoin_FOJ(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
     val sqlQuery =
       """
@@ -262,10 +262,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -277,7 +277,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_BroadcastHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -289,10 +289,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_LOJ(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -304,10 +304,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_ROJ(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -319,10 +319,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_FOJ(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -334,10 +334,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_HashJoin_1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -350,7 +350,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_NestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -362,12 +362,12 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Join_PartialKey(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
     val sqlQuery =
       """
@@ -385,7 +385,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Union(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -400,7 +400,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
     val sqlQuery =
       """
@@ -415,9 +415,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_PartialKey1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
     val sqlQuery =
       """
@@ -432,9 +432,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_PartialKey2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, false)
     val sqlQuery =
       """
@@ -449,9 +449,9 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_PartialKey3(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
     val sqlQuery =
       """
@@ -466,7 +466,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_Singleton1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
     val sqlQuery =
       """
@@ -481,7 +481,7 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Rank_Singleton2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
     val sqlQuery =
       """
@@ -496,10 +496,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Correlate1(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
@@ -513,10 +513,10 @@ class RemoveShuffleTest extends TableTestBase {
 
   @Test
   def testRemoveHashShuffle_Correlate2(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
@@ -531,10 +531,10 @@ class RemoveShuffleTest extends TableTestBase {
   @Test
   def testRemoveHashShuffle_Correlate3(): Unit = {
     // do not remove shuffle
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     util.addFunction("split", new TableFunc1)
     val sqlQuery =
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala
index d1821d6..6998381 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SetOperatorsTest.scala
@@ -34,7 +34,7 @@ class SetOperatorsTest extends TableTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_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-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala
index 5bfc921..c9bfcc4 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SinkTest.scala
@@ -44,7 +44,7 @@ class SinkTest extends TableTestBase {
 
   @Test
   def testMultiSinks(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
     util.tableEnv.registerTable("table1", table1)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala
index f0c43b1..72ec274 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortLimitTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.batch.sql
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule
+import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
 import org.apache.flink.table.util.TableTestBase
 
 import org.junit.Test
@@ -30,71 +30,66 @@ class SortLimitTest extends TableTestBase {
 
   private val util = batchTestUtil()
   util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
-  util.tableEnv.getConfig.getConf.setInteger(
+  util.tableEnv.getConfig.getConfiguration.setInteger(
     ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, 200)
 
   @Test
   def testNonRangeSortWithoutOffset(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5")
   }
 
   @Test
   def testNonRangeSortWithLimit0(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0")
   }
 
   @Test
   def testNonRangeSortOnlyWithOffset(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS")
   }
 
   @Test
   def testNoneRangeSortWithOffsetLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1")
   }
 
   @Test
   def testNoneRangeSortWithOffsetLimit0(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1")
   }
 
   @Test
   def testRangeSortOnWithoutOffset(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 5")
   }
 
   @Test
   def testRangeSortOnWithLimit0(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC LIMIT 0")
   }
 
   @Test
   def testRangeSortOnlyWithOffset(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC OFFSET 10 ROWS")
   }
 
   @Test
   def testRangeSortWithOffsetLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 10 OFFSET 1")
   }
 
   @Test
   def testRangeSortWithOffsetLimit0(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b LIMIT 0 OFFSET 1")
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala
index 801f35c..22f4908 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SortTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.batch.sql
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule
+import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
 import org.apache.flink.table.util.TableTestBase
 
 import org.junit.Test
@@ -33,43 +33,42 @@ class SortTest extends TableTestBase {
 
   @Test
   def testNonRangeSortOnSingleFieldWithoutForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
-    util.tableEnv.getConfig.getConf.setInteger(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, -1)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
 
   @Test
   def testNonRangeSortOnMultiFieldsWithoutForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
-    util.tableEnv.getConfig.getConf.setInteger(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
+      SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, -1)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC, b")
   }
 
   @Test
   def testNonRangeSortWithForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
-      BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, false)
-    util.tableEnv.getConfig.getConf.setInteger(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
+      SQL_EXEC_SORT_RANGE_ENABLED, false)
+    util.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, 200)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
 
   @Test
   def testRangeSortWithoutForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setInteger(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, -1)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
 
   @Test
   def testRangeSortWithForceLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setInteger(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_EXEC_SORT_DEFAULT_LIMIT, 200)
     util.verifyPlan("SELECT * FROM MyTable ORDER BY a DESC")
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala
index 7169f7a..7288a5b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/SubplanReuseTest.scala
@@ -20,10 +20,12 @@ package org.apache.flink.table.plan.batch.sql
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, OperatorType, OptimizerConfigOptions}
+import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
 import org.apache.flink.table.functions.aggfunctions.FirstValueAggFunction.IntFirstValueAggFunction
 import org.apache.flink.table.functions.aggfunctions.LastValueAggFunction.LongLastValueAggFunction
-import org.apache.flink.table.plan.rules.physical.batch.{BatchExecSortMergeJoinRule, BatchExecSortRule}
+import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortMergeJoinRule
+import org.apache.flink.table.plan.rules.physical.batch.BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED
+import org.apache.flink.table.plan.util.OperatorType
 import org.apache.flink.table.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf
 import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.{NonDeterministicTableFunc, StringSplit}
 import org.apache.flink.table.util.TableTestBase
@@ -36,9 +38,9 @@ class SubplanReuseTest extends TableTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
     util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f)
@@ -46,7 +48,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testDisableSubplanReuse(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
     val sqlQuery =
       """
@@ -60,7 +62,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseWithDifferentRowType(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
     // can not reuse because of different row-type
     val sqlQuery =
@@ -74,7 +76,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testEnableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
     val sqlQuery =
       """
@@ -86,7 +88,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testDisableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
     val sqlQuery =
       """
@@ -99,9 +101,9 @@ class SubplanReuseTest extends TableTestBase {
   @Test
   def testSubplanReuseOnSourceWithLimit(): Unit = {
     // TODO re-check this plan after PushLimitIntoTableSourceScanRule is introduced
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -125,7 +127,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnCalc(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -165,7 +167,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnExchange(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -179,7 +181,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnHashAggregate(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
     val sqlQuery =
       """
@@ -191,7 +193,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnSortAggregate(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
     val sqlQuery =
       """
@@ -217,7 +219,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnSort(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    util.tableEnv.getConfig.getConfiguration.setBoolean(SQL_EXEC_SORT_RANGE_ENABLED, true)
     val sqlQuery =
       """
         |WITH r AS (SELECT c, SUM(a) a, SUM(b) b FROM x GROUP BY c ORDER BY a, b DESC)
@@ -228,7 +230,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -271,9 +273,9 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnSortMergeJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecSortMergeJoinRule.SQL_OPTIMIZER_SMJ_REMOVE_SORT_ENABLED, true)
     val sqlQuery =
       """
@@ -285,7 +287,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -297,7 +299,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnNestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -345,7 +347,7 @@ class SubplanReuseTest extends TableTestBase {
   @Test
   def testSubplanReuseOnCorrelate(): Unit = {
     util.addFunction("str_split", new StringSplit())
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -384,7 +386,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testNestedSubplanReuse(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin,SortAgg")
     val sqlQuery =
       """
@@ -420,7 +422,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testBreakupDeadlockOnHashJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "NestedLoopJoin,SortMergeJoin")
     val sqlQuery =
       """
@@ -432,7 +434,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testBreakupDeadlockOnNestedLoopJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala
index bfff146..2f1d077 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/HashAggregateTest.scala
@@ -17,8 +17,9 @@
  */
 package org.apache.flink.table.plan.batch.sql.agg
 
-import org.apache.flink.table.api.AggPhaseEnforcer.AggPhaseEnforcer
-import org.apache.flink.table.api.{AggPhaseEnforcer, OperatorType, OptimizerConfigOptions, ExecutionConfigOptions, TableException, ValidationException}
+import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions, TableException}
+import org.apache.flink.table.plan.util.OperatorType
+import org.apache.flink.table.util.AggregatePhaseStrategy
 
 import org.junit.Before
 import org.junit.runner.RunWith
@@ -29,14 +30,14 @@ import java.util
 import scala.collection.JavaConversions._
 
 @RunWith(classOf[Parameterized])
-class HashAggregateTest(aggStrategy: AggPhaseEnforcer) extends AggregateTestBase {
+class HashAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTestBase {
 
   @Before
   def before(): Unit = {
     // disable sort agg
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
   }
 
@@ -62,11 +63,11 @@ class HashAggregateTest(aggStrategy: AggPhaseEnforcer) extends AggregateTestBase
 object HashAggregateTest {
 
   @Parameterized.Parameters(name = "aggStrategy={0}")
-  def parameters(): util.Collection[AggPhaseEnforcer] = {
-    Seq[AggPhaseEnforcer](
-      AggPhaseEnforcer.AUTO,
-      AggPhaseEnforcer.ONE_PHASE,
-      AggPhaseEnforcer.TWO_PHASE
+  def parameters(): util.Collection[AggregatePhaseStrategy] = {
+    Seq[AggregatePhaseStrategy](
+      AggregatePhaseStrategy.AUTO,
+      AggregatePhaseStrategy.ONE_PHASE,
+      AggregatePhaseStrategy.TWO_PHASE
     )
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala
index f83fc81..0bbed95 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/SortAggregateTest.scala
@@ -17,8 +17,9 @@
  */
 package org.apache.flink.table.plan.batch.sql.agg
 
-import org.apache.flink.table.api.AggPhaseEnforcer.AggPhaseEnforcer
-import org.apache.flink.table.api.{AggPhaseEnforcer, OperatorType, OptimizerConfigOptions, ExecutionConfigOptions}
+import org.apache.flink.table.api.{ExecutionConfigOptions, OptimizerConfigOptions}
+import org.apache.flink.table.plan.util.OperatorType
+import org.apache.flink.table.util.AggregatePhaseStrategy
 
 import org.junit.Before
 import org.junit.runner.RunWith
@@ -29,14 +30,14 @@ import java.util
 import scala.collection.JavaConversions._
 
 @RunWith(classOf[Parameterized])
-class SortAggregateTest(aggStrategy: AggPhaseEnforcer) extends AggregateTestBase {
+class SortAggregateTest(aggStrategy: AggregatePhaseStrategy) extends AggregateTestBase {
 
   @Before
   def before(): Unit = {
     // disable hash agg
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.HashAgg.toString)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
   }
 }
@@ -44,11 +45,11 @@ class SortAggregateTest(aggStrategy: AggPhaseEnforcer) extends AggregateTestBase
 object SortAggregateTest {
 
   @Parameterized.Parameters(name = "aggStrategy={0}")
-  def parameters(): util.Collection[AggPhaseEnforcer] = {
-    Seq[AggPhaseEnforcer](
-      AggPhaseEnforcer.AUTO,
-      AggPhaseEnforcer.ONE_PHASE,
-      AggPhaseEnforcer.TWO_PHASE
+  def parameters(): util.Collection[AggregatePhaseStrategy] = {
+    Seq[AggregatePhaseStrategy](
+      AggregatePhaseStrategy.AUTO,
+      AggregatePhaseStrategy.ONE_PHASE,
+      AggregatePhaseStrategy.TWO_PHASE
     )
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala
index 8405994..52bfff5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/agg/WindowAggregateTest.scala
@@ -18,11 +18,10 @@
 package org.apache.flink.table.plan.batch.sql.agg
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.AggPhaseEnforcer.AggPhaseEnforcer
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions, TableException, ValidationException}
+import org.apache.flink.table.api.{OptimizerConfigOptions, TableException, ValidationException}
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.WeightedAvgWithMerge
-import org.apache.flink.table.util.{CountAggFunction, TableTestBase}
+import org.apache.flink.table.util.{AggregatePhaseStrategy, CountAggFunction, TableTestBase}
 
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
@@ -34,13 +33,13 @@ import java.util
 import scala.collection.JavaConversions._
 
 @RunWith(classOf[Parameterized])
-class WindowAggregateTest(aggStrategy: AggPhaseEnforcer) extends TableTestBase {
+class WindowAggregateTest(aggStrategy: AggregatePhaseStrategy) extends TableTestBase {
 
   private val util = batchTestUtil()
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggStrategy.toString)
     util.addFunction("countFun", new CountAggFunction)
     util.addTableSource[(Int, Timestamp, Int, Long)]("MyTable", 'a, 'b, 'c, 'd)
@@ -305,11 +304,11 @@ class WindowAggregateTest(aggStrategy: AggPhaseEnforcer) extends TableTestBase {
 object WindowAggregateTest {
 
   @Parameterized.Parameters(name = "aggStrategy={0}")
-  def parameters(): util.Collection[AggPhaseEnforcer] = {
-    Seq[AggPhaseEnforcer](
-      AggPhaseEnforcer.AUTO,
-      AggPhaseEnforcer.ONE_PHASE,
-      AggPhaseEnforcer.TWO_PHASE
+  def parameters(): util.Collection[AggregatePhaseStrategy] = {
+    Seq[AggregatePhaseStrategy](
+      AggregatePhaseStrategy.AUTO,
+      AggregatePhaseStrategy.ONE_PHASE,
+      AggregatePhaseStrategy.TWO_PHASE
     )
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala
index 22e3178..756a151 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashJoinTest.scala
@@ -26,9 +26,9 @@ class BroadcastHashJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, ShuffleHashJoin")
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
index f12d487..69b93d5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/BroadcastHashSemiAntiJoinTest.scala
@@ -26,9 +26,9 @@ class BroadcastHashSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, Long.MaxValue)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, ShuffleHashJoin")
     // the result plan may contains NestedLoopJoin (singleRowJoin)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala
index d9067a43..10ea8a3 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/LookupJoinTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.batch.sql.join
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.FlinkBatchProgram
 import org.apache.flink.table.plan.stream.sql.join.TestTemporalTable
 import org.apache.flink.table.util.TableTestBase
@@ -108,21 +107,17 @@ class LookupJoinTest extends TableTestBase {
          |FROM ($sql2) AS T
          |GROUP BY b
       """.stripMargin
-    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConf)
+    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConfiguration)
     programs.remove(FlinkBatchProgram.PHYSICAL)
-    val calciteConfig = CalciteConfig.createBuilder(testUtil.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    testUtil.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    testUtil.replaceBatchProgram(programs)
     testUtil.verifyPlan(sql)
   }
 
   @Test
   def testLogicalPlanWithImplicitTypeCast(): Unit = {
-    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConf)
+    val programs = FlinkBatchProgram.buildProgram(testUtil.tableEnv.getConfig.getConfiguration)
     programs.remove(FlinkBatchProgram.PHYSICAL)
-    val calciteConfig = CalciteConfig.createBuilder(testUtil.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    testUtil.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    testUtil.replaceBatchProgram(programs)
 
     thrown.expect(classOf[TableException])
     thrown.expectMessage("VARCHAR(2147483647) and INTEGER does not have common type now")
@@ -228,7 +223,7 @@ class LookupJoinTest extends TableTestBase {
 
   @Test
   def testReusing(): Unit = {
-    testUtil.tableEnv.getConfig.getConf.setBoolean(
+    testUtil.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
     val sql1 =
       """
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala
index a27cd83..8df75db 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopJoinTest.scala
@@ -26,7 +26,7 @@ class NestedLoopJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
index e281b7c..d152961 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/NestedLoopSemiAntiJoinTest.scala
@@ -26,7 +26,7 @@ class NestedLoopSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin, HashJoin")
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala
index c97787f..e4703f7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashJoinTest.scala
@@ -26,7 +26,7 @@ class ShuffledHashJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, BroadcastHashJoin")
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
index f992753..0089486 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/ShuffledHashSemiAntiJoinTest.scala
@@ -26,7 +26,7 @@ class ShuffledHashSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,
       "SortMergeJoin, NestedLoopJoin, BroadcastHashJoin")
     // the result plan may contains NestedLoopJoin (singleRowJoin)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala
index f780c82..5a97ee7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeJoinTest.scala
@@ -26,7 +26,7 @@ class SortMergeJoinTest extends JoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
index e914160..f676f03 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/batch/sql/join/SortMergeSemiAntiJoinTest.scala
@@ -26,7 +26,7 @@ class SortMergeSemiAntiJoinTest extends SemiAntiJoinTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_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-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala
index 128ed21..f6c1665 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/AggregateReduceGroupingTestBase.scala
@@ -19,7 +19,6 @@ package org.apache.flink.table.plan.common
 
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.Types
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.FlinkBatchProgram
 import org.apache.flink.table.plan.rules.logical.FlinkAggregateRemoveRule
 import org.apache.flink.table.plan.stats.{FlinkStatistic, TableStats}
@@ -69,14 +68,11 @@ abstract class AggregateReduceGroupingTestBase extends TableTestBase {
 
   @Test
   def testAggWithoutAggCall(): Unit = {
-    val programs = util.tableEnv.getConfig.getCalciteConfig.getBatchProgram
-      .getOrElse(FlinkBatchProgram.buildProgram(util.getTableEnv.getConfig.getConf))
+    val programs = util.getBatchProgram()
     programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL)
-      .get.remove(RuleSets.ofList(FlinkAggregateRemoveRule.INSTANCE)) // to prevent the agg from
-    // removing
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+      .get.remove(RuleSets.ofList(FlinkAggregateRemoveRule.INSTANCE))
+    // to prevent the agg from removing
+    util.replaceBatchProgram(programs)
     util.verifyPlan("SELECT a1, b1, c1 FROM T1 GROUP BY a1, b1, c1")
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala
index a883cc2..32cba57 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/common/JoinReorderTestBase.scala
@@ -67,7 +67,7 @@ abstract class JoinReorderTestBase extends TableTestBase {
         "b5" -> new ColumnStats(200L, 0L, 8.0, 8, null, null)
       ))).build())
 
-    util.getTableEnv.getConfig.getConf.setBoolean(
+    util.getTableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_JOIN_REORDER_ENABLED, true)
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/SelectivityEstimatorTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/SelectivityEstimatorTest.scala
index 0503f2d..8494ae1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/SelectivityEstimatorTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/metadata/SelectivityEstimatorTest.scala
@@ -79,7 +79,7 @@ class SelectivityEstimatorTest {
   private def mockScan(
       statistic: FlinkStatistic = FlinkStatistic.UNKNOWN,
       isFilterPushedDown: Boolean = false,
-      tableConfig: TableConfig = TableConfig.DEFAULT): TableScan = {
+      tableConfig: TableConfig = TableConfig.getDefault): TableScan = {
     val tableScan = mock(classOf[TableScan])
     val cluster = mock(classOf[RelOptCluster])
     val planner = mock(classOf[AbstractRelOptPlanner])
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
index e614b2c..6aff288 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/nodes/resource/ExecNodeResourceTest.scala
@@ -78,7 +78,7 @@ class ExecNodeResourceTest(isBatch: Boolean) extends TableTestBase {
 
   @Test
   def testConfigSourceParallelism(): Unit = {
-    testUtil.tableEnv.getConfig.getConf.setInteger(
+    testUtil.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_RESOURCE_SOURCE_PARALLELISM, 100)
     val sqlQuery = "SELECT sum(a) as sum_a, c FROM table3 group by c order by c limit 2"
     testUtil.verifyResource(sqlQuery)
@@ -113,7 +113,7 @@ class ExecNodeResourceTest(isBatch: Boolean) extends TableTestBase {
 
   @Test
   def testSinkConfigParallelism(): Unit = {
-    testUtil.tableEnv.getConfig.getConf.setInteger(
+    testUtil.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_RESOURCE_SINK_PARALLELISM,
       25
     )
@@ -129,7 +129,7 @@ class ExecNodeResourceTest(isBatch: Boolean) extends TableTestBase {
 
   @Test
   def testSinkConfigParallelismWhenMax1(): Unit = {
-    testUtil.tableEnv.getConfig.getConf.setInteger(
+    testUtil.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_RESOURCE_SINK_PARALLELISM,
       25
     )
@@ -145,7 +145,7 @@ class ExecNodeResourceTest(isBatch: Boolean) extends TableTestBase {
 
   @Test
   def testSinkConfigParallelismWhenMax2(): Unit = {
-    testUtil.tableEnv.getConfig.getConf.setInteger(
+    testUtil.tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_RESOURCE_SINK_PARALLELISM,
       25
     )
@@ -171,7 +171,7 @@ object ExecNodeResourceTest {
   }
 
   def setResourceConfig(tableConfig: TableConfig): Unit = {
-    tableConfig.getConf.setInteger(
+    tableConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM,
       18)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala
index 7c826cf..e1f4efe 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/AggregateReduceGroupingRuleTest.scala
@@ -17,7 +17,6 @@
  */
 package org.apache.flink.table.plan.rules.logical
 
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.common.AggregateReduceGroupingTestBase
 import org.apache.flink.table.plan.optimize.program.FlinkBatchProgram
 
@@ -34,13 +33,10 @@ class AggregateReduceGroupingRuleTest extends AggregateReduceGroupingTestBase {
     util.buildBatchProgram(FlinkBatchProgram.LOGICAL_REWRITE)
 
     // remove FlinkAggregateRemoveRule to prevent the agg from removing
-    val programs = util.getTableEnv.getConfig.getCalciteConfig.getBatchProgram
-      .getOrElse(FlinkBatchProgram.buildProgram(util.getTableEnv.getConfig.getConf))
+    val programs = util.getBatchProgram()
     programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL).get
       .remove(RuleSets.ofList(FlinkAggregateRemoveRule.INSTANCE))
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     super.setup()
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala
index 8e901b7..15ba626 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/CalcPruneAggregateCallRuleTest.scala
@@ -17,7 +17,6 @@
  */
 package org.apache.flink.table.plan.rules.logical
 
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
@@ -32,8 +31,7 @@ class CalcPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase {
   override def setup(): Unit = {
     super.setup()
     util.buildBatchProgram(FlinkBatchProgram.LOGICAL)
-
-    val programs = util.getTableEnv.getConfig.getCalciteConfig.getBatchProgram.get
+    val programs = util.getBatchProgram()
     programs.addLast("rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
       .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
@@ -47,9 +45,6 @@ class CalcPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase {
         FlinkCalcMergeRule.INSTANCE,
         PruneAggregateCallRule.CALC_ON_AGGREGATE)
       ).build())
-
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ConvertToNotInOrInRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ConvertToNotInOrInRuleTest.scala
index 669d802..907cc16 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ConvertToNotInOrInRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ConvertToNotInOrInRuleTest.scala
@@ -19,7 +19,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program._
 import org.apache.flink.table.util.TableTestBase
 
@@ -43,9 +42,7 @@ class ConvertToNotInOrInRuleTest extends TableTestBase {
         .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
         .add(RuleSets.ofList(ConvertToNotInOrInRule.INSTANCE))
         .build())
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     util.addTableSource[(Int, Long, Float, Double, String)]("MyTable", 'a, 'b, 'c, 'd, 'e)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.scala
index a8397a6..316ff13 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateInnerJoinTransposeRuleTest.scala
@@ -22,7 +22,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.Types
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkGroupProgramBuilder, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.util.TableTestBase
@@ -68,9 +67,7 @@ class FlinkAggregateInnerJoinTransposeRuleTest extends TableTestBase {
             )).build(), "aggregate join transpose")
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(program).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(program)
 
     util.addTableSource[(Int, Int, String)]("T", 'a, 'b, 'c)
     util.addTableSource("T2",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.scala
index 88483ce..186514d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateOuterJoinTransposeRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE, StreamOptimizeContext}
 import org.apache.flink.table.util.TableTestBase
 
@@ -56,9 +55,7 @@ class FlinkAggregateOuterJoinTransposeRuleTest extends TableTestBase {
         ))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceStreamProgram(program).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceStreamProgram(program)
 
     util.addTableSource[(Int, Long, String, Int)]("T", 'a, 'b, 'c, 'd)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala
index ad70c1a..5479dbe 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkAggregateRemoveRuleTest.scala
@@ -21,7 +21,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.Types
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalAggregate, FlinkLogicalCalc, FlinkLogicalExpand, FlinkLogicalJoin, FlinkLogicalSink, FlinkLogicalTableSourceScan, FlinkLogicalValues}
 import org.apache.flink.table.plan.optimize.program._
@@ -80,9 +79,7 @@ class FlinkAggregateRemoveRuleTest extends TableTestBase {
           FlinkLogicalSink.CONVERTER))
         .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL))
         .build())
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     util.addTableSource[(Int, Int, String)]("MyTable1", 'a, 'b, 'c)
     util.addTableSource("MyTable2",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkCalcMergeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkCalcMergeRuleTest.scala
index 70719f2..0c40b2c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkCalcMergeRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkCalcMergeRuleTest.scala
@@ -19,7 +19,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.nodes.FlinkConventions
 import org.apache.flink.table.plan.nodes.logical.{FlinkLogicalCalc, FlinkLogicalTableSourceScan}
 import org.apache.flink.table.plan.optimize.program._
@@ -60,9 +59,7 @@ class FlinkCalcMergeRuleTest extends TableTestBase {
         ))
         .setRequiredOutputTraits(Array(FlinkConventions.LOGICAL))
         .build())
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     util.addTableSource[(Int, Int, String)]("MyTable", 'a, 'b, 'c)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkFilterJoinRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkFilterJoinRuleTest.scala
index dc6d0929..20cc729 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkFilterJoinRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkFilterJoinRuleTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule
@@ -36,7 +36,8 @@ class FlinkFilterJoinRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala
index 7e7cc3f..d44a775 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkJoinPushExpressionsRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -51,9 +50,7 @@ class FlinkJoinPushExpressionsRuleTest extends TableTestBase {
           FlinkJoinPushExpressionsRule.INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     util.addTableSource[(Int, Long, String)]("l", 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String)]("r", 'd, 'e, 'f)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala
index 085b351..c4ceaf5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkLimit0RemoveRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -48,9 +47,7 @@ class FlinkLimit0RemoveRuleTest extends TableTestBase {
           FlinkLimit0RemoveRule.INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkPruneEmptyRulesTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkPruneEmptyRulesTest.scala
index 0d3f106..7e4aaad 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkPruneEmptyRulesTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkPruneEmptyRulesTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -53,9 +52,7 @@ class FlinkPruneEmptyRulesTest extends TableTestBase {
           FlinkPruneEmptyRules.JOIN_RIGHT_INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     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-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala
index 6f2b0a0..7255bf7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinFilterTransposeRuleTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -37,7 +37,8 @@ class FlinkSemiAntiJoinFilterTransposeRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala
index 349a019..a34faa3 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinJoinTransposeRuleTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -36,7 +36,8 @@ class FlinkSemiAntiJoinJoinTransposeRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala
index 3025495..0c1dc55 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/FlinkSemiAntiJoinProjectTransposeRuleTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -37,7 +37,8 @@ class FlinkSemiAntiJoinProjectTransposeRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala
index 59df263..d9dbfc3 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionEqualityTransferRuleTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -38,7 +38,8 @@ class JoinConditionEqualityTransferRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala
index 7b7f257..4497c51 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinConditionTypeCoerceRuleTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -38,7 +38,8 @@ class JoinConditionTypeCoerceRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala
index ee5526c..1e09648 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDependentConditionDerivationRuleTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -38,7 +38,8 @@ class JoinDependentConditionDerivationRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
index ae0894a..3146d31 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/JoinDeriveNullFilterRuleTest.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.table.api.Types
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.plan.stats.{ColumnStats, FlinkStatistic, TableStats}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -41,7 +41,8 @@ class JoinDeriveNullFilterRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
@@ -50,7 +51,7 @@ class JoinDeriveNullFilterRuleTest extends TableTestBase {
         .build()
     )
 
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       JoinDeriveNullFilterRule.SQL_OPTIMIZER_JOIN_NULL_FILTER_THRESHOLD, 2000000)
     util.addTableSource("MyTable1",
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.STRING, Types.INT, Types.LONG),
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRuleTest.scala
index a601853..de2ce2f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/LogicalUnnestRuleTest.scala
@@ -18,10 +18,9 @@
 
 package org.apache.flink.table.plan.rules.logical
 
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.common.UnnestTestBase
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestUtil
+import org.apache.flink.table.util.{BatchTableTestUtil, TableTestUtil}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -45,9 +44,7 @@ class LogicalUnnestRuleTest extends UnnestTestBase {
         .add(RuleSets.ofList(LogicalUnnestRule.INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.getTableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.getTableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.asInstanceOf[BatchTableTestUtil].replaceBatchProgram(programs)
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala
index 01893f3..f882f8c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectPruneAggregateCallRuleTest.scala
@@ -17,8 +17,8 @@
  */
 package org.apache.flink.table.plan.rules.logical
 
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
+import org.apache.flink.table.util.TableConfigUtils
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.rel.rules.ProjectFilterTransposeRule
@@ -33,7 +33,8 @@ class ProjectPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase {
     super.setup()
     util.buildBatchProgram(FlinkBatchProgram.LOGICAL)
 
-    val programs = util.getTableEnv.getConfig.getCalciteConfig.getBatchProgram.get
+    var calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    val programs = calciteConfig.getBatchProgram.get
     programs.addLast("rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
@@ -44,8 +45,6 @@ class ProjectPruneAggregateCallRuleTest extends PruneAggregateCallRuleTestBase {
           PruneAggregateCallRule.PROJECT_ON_AGGREGATE)
         ).build())
 
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala
index b7a6289..e4ef7e1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ProjectSemiAntiJoinTransposeRuleTest.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.runtime.utils.JavaUserDefinedTableFunctions.StringSplit
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -38,7 +38,8 @@ class ProjectSemiAntiJoinTransposeRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.scala
index 0f59240..6161be5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushFilterIntoTableSourceScanRuleTest.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.table.expressions.utils.Func1
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.{TableTestBase, TestFilterableTableSource}
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase, TestFilterableTableSource}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -34,7 +34,8 @@ class PushFilterIntoTableSourceScanRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_COLLECTION)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
index 60f32e8..4b207f4 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.table.api.{DataTypes, TableSchema, Types}
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.types.TypeInfoDataTypeConverter
-import org.apache.flink.table.util.{TableTestBase, TestNestedProjectableTableSource, TestProjectableTableSource}
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase, TestNestedProjectableTableSource, TestProjectableTableSource}
 import org.apache.flink.types.Row
 
 import org.apache.calcite.plan.hep.HepMatchOrder
@@ -40,7 +40,8 @@ class PushProjectIntoTableSourceScanRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "rules",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala
index 725e135..e5aa8ba 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceIntersectWithSemiJoinRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -46,10 +45,7 @@ class ReplaceIntersectWithSemiJoinRuleTest extends TableTestBase {
         .add(RuleSets.ofList(ReplaceIntersectWithSemiJoinRule.INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
-
+    util.replaceBatchProgram(programs)
 
     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-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala
index 2464bbc..264b4d7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/ReplaceMinusWithAntiJoinRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -46,10 +45,7 @@ class ReplaceMinusWithAntiJoinRuleTest extends TableTestBase {
         .add(RuleSets.ofList(ReplaceMinusWithAntiJoinRule.INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
-
+    util.replaceBatchProgram(programs)
 
     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-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteCoalesceRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteCoalesceRuleTest.scala
index af667b2..be6636f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteCoalesceRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteCoalesceRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.codegen.CodeGenException
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.plan.rules.FlinkBatchRuleSets
@@ -47,10 +46,7 @@ class RewriteCoalesceRuleTest extends TableTestBase {
         .setHepMatchOrder(HepMatchOrder.BOTTOM_UP)
         .add(FlinkBatchRuleSets.DEFAULT_REWRITE_RULES)
         .build())
-
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     util.addTableSource[(Int, String, String, Int, Date, Double, Double, Int)](
       "scott_emp", 'empno, 'ename, 'job, 'mgr, 'hiredate, 'sal, 'comm, 'deptno)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteIntersectAllRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteIntersectAllRuleTest.scala
index 01db7cb..bd6ba00 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteIntersectAllRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteIntersectAllRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -46,10 +45,7 @@ class RewriteIntersectAllRuleTest extends TableTestBase {
         .add(RuleSets.ofList(RewriteIntersectAllRule.INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
-
+    util.replaceBatchProgram(programs)
 
     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-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMinusAllRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMinusAllRuleTest.scala
index 6d5bc0c..0fe095f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMinusAllRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMinusAllRuleTest.scala
@@ -20,7 +20,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -46,10 +45,7 @@ class RewriteMinusAllRuleTest extends TableTestBase {
         .add(RuleSets.ofList(RewriteMinusAllRule.INSTANCE))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
-
+    util.replaceBatchProgram(programs)
 
     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-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala
index e0fdc8d..3f7ce14 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/RewriteMultiJoinConditionRuleTest.scala
@@ -19,7 +19,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkGroupProgramBuilder, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -62,9 +61,7 @@ class RewriteMultiJoinConditionRuleTest extends TableTestBase {
           .build(), "RewriteMultiJoinConditionRule")
         .build())
 
-    val builder = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(program)
-    util.tableEnv.getConfig.setCalciteConfig(builder.build())
+    util.replaceBatchProgram(program)
 
     util.addTableSource[(Int, Long)]("A", 'a1, 'a2)
     util.addTableSource[(Int, Long)]("B", 'b1, 'b2)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyFilterConditionRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyFilterConditionRuleTest.scala
index 7557f98..a65c703 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyFilterConditionRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyFilterConditionRuleTest.scala
@@ -19,7 +19,6 @@ package org.apache.flink.table.plan.rules.logical
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkChainedProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.util.TableTestBase
 
@@ -45,9 +44,7 @@ class SimplifyFilterConditionRuleTest extends TableTestBase {
         .add(RuleSets.ofList(SimplifyFilterConditionRule.EXTENDED))
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
-      .replaceBatchProgram(programs).build()
-    util.tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    util.replaceBatchProgram(programs)
 
     util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyJoinConditionRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyJoinConditionRuleTest.scala
index 67a8c39..c3edfa1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyJoinConditionRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SimplifyJoinConditionRuleTest.scala
@@ -20,7 +20,7 @@ package org.apache.flink.table.plan.rules.logical
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.tools.RuleSets
@@ -36,7 +36,8 @@ class SimplifyJoinConditionRuleTest extends TableTestBase {
   @Before
   def setup(): Unit = {
     util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-    util.tableEnv.getConfig.getCalciteConfig.getBatchProgram.get.addLast(
+    val calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+    calciteConfig.getBatchProgram.get.addLast(
       "SimplifyJoinConditionRule",
       FlinkHepRuleSetProgramBuilder.newBuilder
         .setHepRulesExecutionType(HEP_RULES_EXECUTION_TYPE.RULE_SEQUENCE)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala
index 6469e50..75dc85e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/SplitAggregateRuleTest.scala
@@ -34,7 +34,7 @@ class SplitAggregateRuleTest extends TableTestBase {
   util.addTableSource[(Long, Int, String)]("MyTable", 'a, 'b, 'c)
   util.buildStreamProgram(FlinkStreamProgram.PHYSICAL)
   util.enableMiniBatch()
-  util.tableEnv.getConfig.getConf.setBoolean(
+  util.tableEnv.getConfig.getConfiguration.setBoolean(
     OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
 
   @Test
@@ -178,7 +178,7 @@ class SplitAggregateRuleTest extends TableTestBase {
 
   @Test
   def testBucketsConfiguration(): Unit = {
-    util.tableEnv.getConfig.getConf.setInteger(
+    util.tableEnv.getConfig.getConfiguration.setInteger(
       OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_BUCKET_NUM, 100)
     val sqlQuery = "SELECT COUNT(DISTINCT c) FROM MyTable"
     util.verifyPlan(sqlQuery)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubQueryTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubQueryTestBase.scala
index a97ff2d..1fc7d6f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubQueryTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/logical/subquery/SubQueryTestBase.scala
@@ -30,7 +30,8 @@ class SubQueryTestBase extends TableTestBase {
   protected val util: BatchTableTestUtil = batchTestUtil()
 
   util.buildBatchProgram(FlinkBatchProgram.DEFAULT_REWRITE)
-  val builder = CalciteConfig.createBuilder(util.tableEnv.getConfig.getCalciteConfig)
+  var calciteConfig = TableConfigUtils.getCalciteConfig(util.tableEnv.getConfig)
+  val builder = CalciteConfig.createBuilder(calciteConfig)
   builder.replaceSqlToRelConverterConfig(
     SqlToRelConverter.configBuilder()
       .withTrimUnusedFields(false)
@@ -39,5 +40,5 @@ class SubQueryTestBase extends TableTestBase {
       .withInSubQueryThreshold(3)
       .build())
 
-  util.tableEnv.getConfig.setCalciteConfig(builder.build())
+  util.tableEnv.getConfig.setPlannerConfig(builder.build())
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
index 3d4d694..03564ba 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalHashAggRuleTest.scala
@@ -39,10 +39,10 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase {
 
   @Test
   def testRemoveRedundantLocalHashAgg_ShuffleKeyFromJoin(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,SortAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -54,9 +54,9 @@ class RemoveRedundantLocalHashAggRuleTest extends TableTestBase {
 
   @Test
   def testRemoveRedundantLocalHashAgg_ShuffleKeyFromRank(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       BatchExecJoinRuleBase.SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED, true)
     val sqlQuery =
       """
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
index e3806b4..775de9d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/batch/RemoveRedundantLocalSortAggRuleTest.scala
@@ -39,10 +39,10 @@ class RemoveRedundantLocalSortAggRuleTest extends TableTestBase {
 
   @Test
   def testRemoveRedundantLocalSortAggWithSort(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortMergeJoin,NestedLoopJoin,HashAgg")
     // disable BroadcastHashJoin
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
     val sqlQuery =
       """
@@ -54,7 +54,7 @@ class RemoveRedundantLocalSortAggRuleTest extends TableTestBase {
 
   @Test
   def testRemoveRedundantLocalSortAggWithoutSort(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,NestedLoopJoin,HashAgg")
     val sqlQuery =
       """
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
index b295700..d5c3408 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/rules/physical/stream/RetractionRulesWithTwoStageAggTest.scala
@@ -21,8 +21,8 @@ package org.apache.flink.table.plan.rules.physical.stream
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.api.OptimizerConfigOptions
+import org.apache.flink.table.util.{AggregatePhaseStrategy, TableTestBase}
 
 import org.junit.{Before, Test}
 
@@ -34,9 +34,10 @@ class RetractionRulesWithTwoStageAggTest extends TableTestBase {
   @Before
   def before(): Unit = {
     util.enableMiniBatch()
-    util.tableEnv.getConfig.withIdleStateRetentionTime(Time.hours(1))
-    util.tableEnv.getConfig.getConf.setString(
-      OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, AggPhaseEnforcer.TWO_PHASE.toString)
+    util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
+    util.tableEnv.getConfig.getConfiguration.setString(
+      OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
+      AggregatePhaseStrategy.TWO_PHASE.toString)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala
index aaf815e..1e26480 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/DagOptimizationTest.scala
@@ -120,7 +120,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testSingleSinkSplitOnUnion(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val sqlQuery = "SELECT SUM(a) AS total_sum FROM " +
@@ -134,7 +134,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks1(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS sum_a, c FROM MyTable GROUP BY c")
     util.tableEnv.registerTable("table1", table1)
@@ -152,7 +152,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks2(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
 
@@ -173,7 +173,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks3(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
 
@@ -217,7 +217,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinks5(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     // test with non-deterministic udf
     util.tableEnv.registerFunction("random_udf", new NonDeterministicUdf())
@@ -237,7 +237,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksWithUDTF(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addFunction("split", new TableFunc1)
     val sqlQuery1 =
@@ -276,7 +276,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion1(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table = util.tableEnv.sqlQuery(
@@ -296,9 +296,9 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion2(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
@@ -331,7 +331,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion3(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
@@ -359,7 +359,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiSinksSplitOnUnion4(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
 
@@ -453,7 +453,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testMultiLevelViews(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
@@ -489,7 +489,7 @@ class DagOptimizationTest extends TableTestBase {
 
   @Test
   def testSharedUnionNode(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.SQL_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala
index f99ddf7..4d79cfb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/MiniBatchIntervalInferTest.scala
@@ -20,10 +20,11 @@ package org.apache.flink.table.plan.stream.sql
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.{ExecutionConfigOptions, TableConfig}
 import org.apache.flink.table.api.scala._
+import org.apache.flink.table.plan.util.WindowEmitStrategy.{SQL_EXEC_EMIT_EARLY_FIRE_DELAY, SQL_EXEC_EMIT_EARLY_FIRE_ENABLED}
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{TableConfigUtils, TableTestBase}
 
 import org.junit.{Before, Test}
 
@@ -40,13 +41,13 @@ class MiniBatchIntervalInferTest extends TableTestBase {
       "MyTable1", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
     util.addDataStream[(Int, String, Long)](
       "MyTable2", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
   }
 
   @Test
   def testMiniBatchOnly(): Unit = {
-    util.tableEnv.getConfig.getConf
+    util.tableEnv.getConfig.getConfiguration
         .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
     val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable1 GROUP BY b"
     util.verifyPlan(sql)
@@ -54,7 +55,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testRedundantWatermarkDefinition(): Unit = {
-    util.tableEnv.getConfig.getConf
+    util.tableEnv.getConfig.getConfiguration
         .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
     util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
     val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable3 GROUP BY b"
@@ -63,9 +64,10 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testWindowWithEarlyFire(): Unit = {
-    util.tableEnv.getConfig.getConf
+    val tableConfig = util.tableEnv.getConfig
+    tableConfig.getConfiguration
         .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
-    util.tableEnv.getConfig.withEarlyFireInterval(Time.milliseconds(500))
+    withEarlyFireDelay(tableConfig, Time.milliseconds(500))
     util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
     val sql =
       """
@@ -85,7 +87,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testWindowCascade(): Unit = {
-    util.tableEnv.getConfig.getConf
+    util.tableEnv.getConfig.getConfiguration
         .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "3 s")
     util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
     val sql =
@@ -108,7 +110,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   def testWindowJoinWithMiniBatch(): Unit = {
     util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
     util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
     val sql =
@@ -130,7 +132,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   @Test
   def testRowtimeRowsOverWithMiniBatch(): Unit = {
     util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
     val sql =
@@ -153,7 +155,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
     util.addTableWithWatermark("Orders", util.tableEnv.scan("MyTable1"), "rowtime", 0)
     util.addTableWithWatermark("RatesHistory", util.tableEnv.scan("MyTable2"), "rowtime", 0)
 
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
     util.addFunction(
@@ -180,7 +182,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
     // infer result: miniBatchInterval=[Rowtime, 0ms]
     util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
     util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
     val sql =
@@ -206,7 +208,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   def testMultiOperatorNeedsWatermark2(): Unit = {
     util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
     util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
 
     val sql =
@@ -242,7 +244,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   @Test
   def testMultiOperatorNeedsWatermark3(): Unit = {
     util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
 
     val sql =
@@ -276,9 +278,9 @@ class MiniBatchIntervalInferTest extends TableTestBase {
     util.addTableWithWatermark("T3", util.tableEnv.scan("T1"), "rowtime", 0)
     util.addTableWithWatermark("T4", util.tableEnv.scan("T2"), "rowtime", 0)
 
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "500 ms")
-    util.tableEnv.getConfig.getConf.setLong(
+    util.tableEnv.getConfig.getConfiguration.setLong(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 300L)
 
     val table1 = util.tableEnv.sqlQuery(
@@ -333,4 +335,17 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
     util.verifyExplain()
   }
+
+  private def withEarlyFireDelay(tableConfig: TableConfig, interval: Time): Unit = {
+    val intervalInMillis = interval.toMilliseconds
+    val preEarlyFireInterval = TableConfigUtils.getMillisecondFromConfigDuration(
+      tableConfig, SQL_EXEC_EMIT_EARLY_FIRE_DELAY)
+    if (preEarlyFireInterval != null && (preEarlyFireInterval != intervalInMillis)) { //
+      // earlyFireInterval of the two query config is not equal and not the default
+      throw new RuntimeException("Currently not support different earlyFireInterval configs in " +
+        "one job")
+    }
+    tableConfig.getConfiguration.setBoolean(SQL_EXEC_EMIT_EARLY_FIRE_ENABLED, true)
+    tableConfig.getConfiguration.setString(SQL_EXEC_EMIT_EARLY_FIRE_DELAY, intervalInMillis + " ms")
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala
index f43e6d9..66705e0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/ModifiedMonotonicityTest.scala
@@ -65,9 +65,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
 
   @Test
   def testMaxWithRetractOptimizeWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf
+    util.tableEnv.getConfig.getConfiguration
       .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
     val query = "SELECT a1, max(a3) from (SELECT a1, a2, max(a3) as a3 FROM A GROUP BY a1, a2) " +
       "group by a1"
@@ -76,9 +76,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
 
   @Test
   def testMinWithRetractOptimizeWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf
+    util.tableEnv.getConfig.getConfiguration
       .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
     val query = "SELECT min(a3) from (SELECT a1, a2, min(a3) as a3 FROM A GROUP BY a1, a2)"
     util.verifyPlanWithTrait(query)
@@ -86,9 +86,9 @@ class ModifiedMonotonicityTest extends TableTestBase {
 
   @Test
   def testMinCanNotOptimizeWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf
+    util.tableEnv.getConfig.getConfiguration
       .setString(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "100 ms")
     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-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala
index 8a455b7..53aa95c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/SubplanReuseTest.scala
@@ -35,9 +35,9 @@ class SubplanReuseTest extends TableTestBase {
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
     util.addTableSource[(Int, Long, String)]("x", 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String)]("y", 'd, 'e, 'f)
@@ -45,7 +45,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testDisableSubplanReuse(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, false)
     val sqlQuery =
       """
@@ -59,7 +59,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseWithDifferentRowType(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
     // can not reuse because of different row-type
     val sqlQuery =
@@ -73,7 +73,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testEnableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, true)
     val sqlQuery =
       """
@@ -85,7 +85,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testDisableReuseTableSource(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_REUSE_TABLE_SOURCE_ENABLED, false)
     val sqlQuery =
       """
@@ -181,7 +181,7 @@ class SubplanReuseTest extends TableTestBase {
 
   @Test
   def testSubplanReuseOnLimit(): Unit = {
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin,SortMergeJoin")
     val sqlQuery =
       """
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
index a4db880..0cca30a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/AggregateTest.scala
@@ -62,18 +62,18 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testAggWithMiniBatch(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
     util.verifyPlan("SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c)  FROM MyTable GROUP BY b")
   }
 
   @Test
   def testAggAfterUnionWithMiniBatch(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
     val query =
       """
@@ -95,9 +95,9 @@ class AggregateTest extends TableTestBase {
   @Test
   def testLocalGlobalAggAfterUnion(): Unit = {
     // enable local global optimize
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
     val sql =
@@ -128,9 +128,9 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testAggWithFilterClauseWithLocalGlobal(): Unit = {
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
     val sql =
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala
index 5d9a05b..8772500 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/DistinctAggregateTest.scala
@@ -20,11 +20,10 @@ package org.apache.flink.table.plan.stream.sql.agg
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.AggPhaseEnforcer.AggPhaseEnforcer
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions}
+import org.apache.flink.table.api.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.rules.physical.stream.IncrementalAggregateRule
-import org.apache.flink.table.util.{StreamTableTestUtil, TableTestBase}
+import org.apache.flink.table.util.{AggregatePhaseStrategy, StreamTableTestUtil, TableTestBase}
 
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
@@ -35,7 +34,7 @@ import java.util
 @RunWith(classOf[Parameterized])
 class DistinctAggregateTest(
     splitDistinctAggEnabled: Boolean,
-    aggPhaseEnforcer: AggPhaseEnforcer)
+    aggPhaseEnforcer: AggregatePhaseStrategy)
   extends TableTestBase {
 
   protected val util: StreamTableTestUtil = streamTestUtil()
@@ -43,14 +42,14 @@ class DistinctAggregateTest(
 
   @Before
   def before(): Unit = {
-    util.tableEnv.getConfig.withIdleStateRetentionTime(Time.hours(1))
+    util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
     util.enableMiniBatch()
-    util.tableEnv.getConfig.getConf.setString(
+    util.tableEnv.getConfig.getConfiguration.setString(
       OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, aggPhaseEnforcer.toString)
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, splitDistinctAggEnabled)
     // disable incremental agg
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       IncrementalAggregateRule.SQL_OPTIMIZER_INCREMENTAL_AGG_ENABLED, false)
   }
 
@@ -216,10 +215,10 @@ object DistinctAggregateTest {
   @Parameterized.Parameters(name = "splitDistinctAggEnabled={0}, aggPhaseEnforcer={1}")
   def parameters(): util.Collection[Array[Any]] = {
     util.Arrays.asList(
-      Array(true, AggPhaseEnforcer.ONE_PHASE),
-      Array(true, AggPhaseEnforcer.TWO_PHASE),
-      Array(false, AggPhaseEnforcer.ONE_PHASE),
-      Array(false, AggPhaseEnforcer.TWO_PHASE)
+      Array(true, AggregatePhaseStrategy.ONE_PHASE),
+      Array(true, AggregatePhaseStrategy.TWO_PHASE),
+      Array(false, AggregatePhaseStrategy.ONE_PHASE),
+      Array(false, AggregatePhaseStrategy.TWO_PHASE)
     )
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/IncrementalAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/IncrementalAggregateTest.scala
index 4adad28..ad4d1c4 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/IncrementalAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/IncrementalAggregateTest.scala
@@ -18,9 +18,8 @@
 
 package org.apache.flink.table.plan.stream.sql.agg
 
-import org.apache.flink.table.api.AggPhaseEnforcer.AggPhaseEnforcer
-import org.apache.flink.table.api.AggPhaseEnforcer
 import org.apache.flink.table.plan.rules.physical.stream.IncrementalAggregateRule
+import org.apache.flink.table.util.AggregatePhaseStrategy
 
 import org.junit.Before
 import org.junit.runner.RunWith
@@ -31,14 +30,14 @@ import java.util
 @RunWith(classOf[Parameterized])
 class IncrementalAggregateTest(
     splitDistinctAggEnabled: Boolean,
-    aggPhaseEnforcer: AggPhaseEnforcer)
+    aggPhaseEnforcer: AggregatePhaseStrategy)
   extends DistinctAggregateTest(splitDistinctAggEnabled, aggPhaseEnforcer) {
 
   @Before
   override def before(): Unit = {
     super.before()
     // enable incremental agg
-    util.tableEnv.getConfig.getConf.setBoolean(
+    util.tableEnv.getConfig.getConfiguration.setBoolean(
       IncrementalAggregateRule.SQL_OPTIMIZER_INCREMENTAL_AGG_ENABLED, true)
   }
 }
@@ -47,7 +46,7 @@ object IncrementalAggregateTest {
   @Parameterized.Parameters(name = "splitDistinctAggEnabled={0}, aggPhaseEnforcer={1}")
   def parameters(): util.Collection[Array[Any]] = {
     util.Arrays.asList(
-      Array(true, AggPhaseEnforcer.TWO_PHASE)
+      Array(true, AggregatePhaseStrategy.TWO_PHASE)
     )
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala
index 60816db..5eca0cb 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/plan/stream/sql/agg/TwoStageAggregateTest.scala
@@ -20,9 +20,9 @@ package org.apache.flink.table.plan.stream.sql.agg
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions}
+import org.apache.flink.table.api.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.util.TableTestBase
+import org.apache.flink.table.util.{AggregatePhaseStrategy, TableTestBase}
 
 import org.junit.{Before, Test}
 
@@ -34,9 +34,10 @@ class TwoStageAggregateTest extends TableTestBase {
   @Before
   def before(): Unit = {
     util.enableMiniBatch()
-    util.tableEnv.getConfig.withIdleStateRetentionTime(Time.hours(1))
-    util.tableEnv.getConfig.getConf.setString(
-      OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, AggPhaseEnforcer.TWO_PHASE.toString)
+    util.tableEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
+    util.tableEnv.getConfig.getConfiguration.setString(
+      OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
+      AggregatePhaseStrategy.TWO_PHASE.toString)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
index 14010f5..aebdd74 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/CalcITCase.scala
@@ -1229,8 +1229,8 @@ class CalcITCase extends BatchTestBase {
       new RowTypeInfo(INT_TYPE_INFO, LONG_TYPE_INFO, BYTE_PRIMITIVE_ARRAY_TYPE_INFO),
       "a, b, c",
       nullablesOfNullData3)
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
-    conf.getConf.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+    conf.getConfiguration.setBoolean(BatchExecSortRule.SQL_EXEC_SORT_RANGE_ENABLED, true)
     checkResult(
       "select * from BinaryT order by c",
       nullData3.sortBy((x : Row) =>
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
index fd1b29b..0fc7e2e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/DecimalITCase.scala
@@ -773,7 +773,7 @@ class DecimalITCase extends BatchTestBase {
 
   @Test
   def testJoin1(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
 
     checkQuery1(
@@ -786,7 +786,7 @@ class DecimalITCase extends BatchTestBase {
 
   @Test
   def testJoin2(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
 
     checkQuery1(
@@ -799,7 +799,7 @@ class DecimalITCase extends BatchTestBase {
 
   @Test
   def testJoin3(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
 
     checkQuery1(
@@ -813,7 +813,7 @@ class DecimalITCase extends BatchTestBase {
 
   @Test
   def testJoin4(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
 
     checkQuery1(
@@ -826,7 +826,7 @@ class DecimalITCase extends BatchTestBase {
 
   @Test
   def testJoin5(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
 
     checkQuery1(
@@ -839,7 +839,7 @@ class DecimalITCase extends BatchTestBase {
 
   @Test
   def testJoin6(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
 
     checkQuery1(
@@ -852,7 +852,7 @@ class DecimalITCase extends BatchTestBase {
 
   @Test
   def testJoin7(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
     checkQuery1(
       Seq(DECIMAL(8, 2), DECIMAL(8, 4), INT, DOUBLE),
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala
index bf9b4f6..7fd55e3 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/MiscITCase.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.runtime.batch.sql
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.typeutils.RowTypeInfo
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM
 import org.apache.flink.table.runtime.batch.sql.join.JoinITCaseHelper
 import org.apache.flink.table.runtime.batch.sql.join.JoinType.SortMergeJoin
 import org.apache.flink.table.runtime.utils.BatchTestBase
@@ -417,7 +417,7 @@ class MiscITCase extends BatchTestBase {
 
   @Test
   def testOrderByAgg(): Unit = {
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+    tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
     env.setParallelism(1)
     checkQuery(
       Seq((1, 10), (1, 20), (10, 1), (10, 2)),
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala
index 9964a0e..d9faca9 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/SortLimitITCase.scala
@@ -109,7 +109,7 @@ class SortLimitITCase extends BatchTestBase {
 
   @Test
   def testOrderBehindField(): Unit = {
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
     val expected = data3.sortBy((x : Row) => x.getField(2).asInstanceOf[String])
 
     checkResult(
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
index 59d7232..e8d9082 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/UnionITCase.scala
@@ -48,7 +48,7 @@ class UnionITCase extends BatchTestBase {
     registerCollection("Table3", smallData3, type3, "a, b, c", nullablesOfSmallData3)
     registerCollection("Table5", data5, type5, "d, e, f, g, h", nullablesOfData5)
     registerCollection("Table6", data6, type6, "a, b, c", Array(false, false, false))
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
   }
 
@@ -113,9 +113,9 @@ class UnionITCase extends BatchTestBase {
     */
   @Test
   def testJoinAfterDifferentTypeUnionAll(): Unit = {
-    tEnv.getConfig.getConf.setLong(
+    tEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
     checkResult(
       "SELECT a, c, g FROM (SELECT t1.a, t1.b, t1.c FROM Table3 t1 UNION ALL" +
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
index dda022e..f6aa573 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateJoinTransposeITCase.scala
@@ -19,7 +19,8 @@
 package org.apache.flink.table.runtime.batch.sql.agg
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.{ExecutionConfigOptions, TableException, Types}
+import org.apache.flink.table.api.ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS
+import org.apache.flink.table.api.{TableException, Types}
 import org.apache.flink.table.calcite.CalciteConfig
 import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram, FlinkGroupProgramBuilder, FlinkHepRuleSetProgramBuilder, HEP_RULES_EXECUTION_TYPE}
 import org.apache.flink.table.plan.rules.logical.{AggregateReduceGroupingRule, FlinkAggregateJoinTransposeRule}
@@ -27,6 +28,7 @@ import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.runtime.utils.BatchTestBase
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.runtime.utils.TestData._
+import org.apache.flink.table.util.TableConfigUtils
 
 import org.apache.calcite.plan.hep.HepMatchOrder
 import org.apache.calcite.rel.rules._
@@ -41,7 +43,7 @@ class AggregateJoinTransposeITCase extends BatchTestBase {
   @Before
   override def before(): Unit = {
     super.before()
-    val programs = FlinkBatchProgram.buildProgram(tEnv.getConfig.getConf)
+    val programs = FlinkBatchProgram.buildProgram(tEnv.getConfig.getConfiguration)
     // remove FlinkAggregateJoinTransposeRule from logical program (volcano planner)
     programs.getFlinkRuleSetProgram(FlinkBatchProgram.LOGICAL)
       .getOrElse(throw new TableException(s"${FlinkBatchProgram.LOGICAL} does not exist"))
@@ -71,12 +73,13 @@ class AggregateJoinTransposeITCase extends BatchTestBase {
             )).build(), "aggregate join transpose")
         .build()
     )
-    val calciteConfig = CalciteConfig.createBuilder(tEnv.getConfig.getCalciteConfig)
+    var calciteConfig = TableConfigUtils.getCalciteConfig(tEnv.getConfig)
+    calciteConfig = CalciteConfig.createBuilder(calciteConfig)
       .replaceBatchProgram(programs).build()
-    tEnv.getConfig.setCalciteConfig(calciteConfig)
+    tEnv.getConfig.setPlannerConfig(calciteConfig)
 
     // HashJoin is disabled due to translateToPlanInternal method is not implemented yet
-    tEnv.getConfig.getConf.setString(ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
+    tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
     registerCollection("T3", data3, type3, "a, b, c", nullablesOfData3)
 
     registerCollection("MyTable",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
index bd4c89b..7a4b648 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/AggregateReduceGroupingITCase.scala
@@ -18,7 +18,8 @@
 package org.apache.flink.table.runtime.batch.sql.agg
 
 import org.apache.flink.api.java.typeutils.RowTypeInfo
-import org.apache.flink.table.api.{OptimizerConfigOptions, ExecutionConfigOptions, Types}
+import org.apache.flink.table.api.{OptimizerConfigOptions, Types}
+import org.apache.flink.table.api.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM, SQL_RESOURCE_HASH_AGG_TABLE_MEM}
 import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.runtime.functions.SqlDateTimeUtils.unixTimestampToLocalDateTime
 import org.apache.flink.table.runtime.utils.BatchTestBase
@@ -111,12 +112,12 @@ class AggregateReduceGroupingITCase extends BatchTestBase {
       FlinkStatistic.builder().uniqueKeys(Set(Set("a6").asJava).asJava).build()
     )
     // HashJoin is disabled due to translateToPlanInternal method is not implemented yet
-    tEnv.getConfig.getConf.setString(ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
+    tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "HashJoin")
   }
 
   @Test
   def testSingleAggOnTable_SortAgg(): Unit = {
-    tEnv.getConfig.getConf.setString(ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+    tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
     testSingleAggOnTable()
     checkResult("SELECT a6, b6, max(c6), count(d6), sum(e6) FROM T6 GROUP BY a6, b6",
       (0 until 50000).map(i => row(i, 1L, if (i % 500 == 0) null else s"Hello$i", 1L, 10))
@@ -125,21 +126,21 @@ class AggregateReduceGroupingITCase extends BatchTestBase {
 
   @Test
   def testSingleAggOnTable_HashAgg_WithLocalAgg(): Unit = {
-    tEnv.getConfig.getConf.setString(ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+    tEnv.getConfig.getConfiguration.setString(
       OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, "TWO_PHASE")
     // set smaller parallelism to avoid MemoryAllocationException
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 2)
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
+    tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 2)
+    tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
     testSingleAggOnTable()
   }
 
   @Test
   def testSingleAggOnTable_HashAgg_WithoutLocalAgg(): Unit = {
-    tEnv.getConfig.getConf.setString(ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
+    tEnv.getConfig.getConfiguration.setString(
       OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY, "ONE_PHASE")
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
+    tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
     testSingleAggOnTable()
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala
index efb77b8..296e50d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashAggITCase.scala
@@ -28,7 +28,7 @@ class HashAggITCase
     extends AggregateITCaseBase("HashAggregate") {
 
   override def prepareAggOp(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "SortAgg")
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
index 306a1db..9218e55 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/HashDistinctAggregateITCase.scala
@@ -18,7 +18,8 @@
 
 package org.apache.flink.table.runtime.batch.sql.agg
 
-import org.apache.flink.table.api.{OperatorType, ExecutionConfigOptions}
+import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.plan.util.OperatorType
 
 /**
   * HashDistinctAggITCase using HashAgg Operator.
@@ -26,7 +27,7 @@ import org.apache.flink.table.api.{OperatorType, ExecutionConfigOptions}
 class HashDistinctAggregateITCase extends DistinctAggregateITCaseBase {
 
   override def prepareAggOp(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, OperatorType.SortAgg.toString)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala
index 5ef19fd..f40d0af 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortAggITCase.scala
@@ -21,7 +21,8 @@ package org.apache.flink.table.runtime.batch.sql.agg
 import org.apache.flink.api.common.typeinfo.{BasicArrayTypeInfo, PrimitiveArrayTypeInfo, TypeInformation}
 import org.apache.flink.api.java.typeutils.{MapTypeInfo, ObjectArrayTypeInfo, RowTypeInfo, TupleTypeInfo, TypeExtractor}
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{ExecutionConfigOptions, Types}
+import org.apache.flink.table.api.ExecutionConfigOptions.{SQL_EXEC_DISABLED_OPERATORS, SQL_RESOURCE_DEFAULT_PARALLELISM}
+import org.apache.flink.table.api.Types
 import org.apache.flink.table.functions.AggregateFunction
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.WeightedAvgWithMergeAndReset
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
@@ -43,8 +44,8 @@ import scala.collection.Seq
 class SortAggITCase
     extends AggregateITCaseBase("SortAggregate") {
   override def prepareAggOp(): Unit = {
-    tEnv.getConfig.getConf.setString(
-      ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
+    tEnv.getConfig.getConfiguration.setString(
+      SQL_EXEC_DISABLED_OPERATORS, "HashAgg")
 
     registerFunction("countFun", new CountAggFunction())
     registerFunction("intSumFun", new IntSumAggFunction())
@@ -62,7 +63,7 @@ class SortAggITCase
 
   @Test
   def testBigDataSimpleArrayUDAF(): Unit = {
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+    tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
     registerFunction("simplePrimitiveArrayUdaf", new SimplePrimitiveArrayUdaf())
     registerRange("RangeT", 1000000)
     env.setParallelism(1)
@@ -266,7 +267,7 @@ class SortAggITCase
 
   @Test
   def testArrayUdaf(): Unit = {
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+    tEnv.getConfig.getConfiguration.setInteger(SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
     env.setParallelism(1)
     checkResult(
       "SELECT myPrimitiveArrayUdaf(a, b) FROM Table3",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
index 8362821..78ea69c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/agg/SortDistinctAggregateITCase.scala
@@ -19,8 +19,9 @@
 package org.apache.flink.table.runtime.batch.sql.agg
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{OperatorType, ExecutionConfigOptions}
+import org.apache.flink.table.api.ExecutionConfigOptions
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.WeightedAvgWithMergeAndReset
+import org.apache.flink.table.plan.util.OperatorType
 import org.apache.flink.table.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.util.{CountAggFunction, IntSumAggFunction}
 
@@ -34,7 +35,7 @@ import scala.collection.Seq
 class SortDistinctAggregateITCase extends DistinctAggregateITCaseBase {
 
   override def prepareAggOp(): Unit = {
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS,  OperatorType.HashAgg.toString)
 
     registerFunction("countFun", new CountAggFunction())
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
index bf1fd5d..c8c35aa 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/InnerJoinITCase.scala
@@ -154,9 +154,9 @@ class InnerJoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
   @Test
   def testBigForSpill(): Unit = {
 
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM, 2)
-    tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM, 2)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
 
     val bigData = Random.shuffle(
       bigIntStringData.union(bigIntStringData).union(bigIntStringData).union(bigIntStringData))
@@ -171,10 +171,10 @@ class InnerJoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
   @Test
   def testSortMergeJoinOutputOrder(): Unit = {
     if (expectedJoinType == SortMergeJoin) {
-      tEnv.getConfig.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
+      conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, 1)
       env.getConfig.setParallelism(1)
 
-      conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
+      conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
 
       val bigData = Random.shuffle(
         bigIntStringData.union(bigIntStringData).union(bigIntStringData).union(bigIntStringData))
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
index dd562f2..f4544f1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCase.scala
@@ -755,7 +755,7 @@ class JoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
   @Ignore
   @Test
   def testJoinCollation(): Unit = {
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
     checkResult(
       """
         |WITH v1 AS (
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala
index aa562d2..de9cb7c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/batch/sql/join/JoinITCaseHelper.scala
@@ -27,7 +27,7 @@ import org.apache.flink.table.runtime.batch.sql.join.JoinType.{BroadcastHashJoin
 object JoinITCaseHelper {
 
   def disableBroadcastHashJoin(tEnv: TableEnvironment): Unit = {
-    tEnv.getConfig.getConf.setLong(
+    tEnv.getConfig.getConfiguration.setLong(
       OptimizerConfigOptions.SQL_OPTIMIZER_BROADCAST_JOIN_THRESHOLD, -1)
   }
 
@@ -40,7 +40,7 @@ object JoinITCaseHelper {
       case SortMergeJoin => "HashJoin, NestedLoopJoin"
       case NestedLoopJoin => "HashJoin, SortMergeJoin"
     }
-    tEnv.getConfig.getConf.setString(
+    tEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_DISABLED_OPERATORS, disabledOperators)
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
index 7392b74..85de55a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/harness/OverWindowHarnessTest.scala
@@ -20,19 +20,20 @@ package org.apache.flink.table.runtime.harness
 
 import java.lang.{Long => JLong}
 import java.util.concurrent.ConcurrentLinkedQueue
-
 import org.apache.flink.api.scala._
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord
-import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.{EnvironmentSettings, StreamQueryConfig, TableConfig, Types}
 import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.scala.internal.StreamTableEnvironmentImpl
 import org.apache.flink.table.runtime.util.StreamRecordUtils.baserow
 import org.apache.flink.table.runtime.util.StreamRecordUtils.binaryrow
 import org.apache.flink.table.runtime.util.BaseRowHarnessAssertor
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
 import org.apache.flink.types.Row
-import org.junit.Test
+
+import org.junit.{Before, Test}
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 
@@ -41,6 +42,14 @@ import scala.collection.mutable
 @RunWith(classOf[Parameterized])
 class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode) {
 
+  @Before
+  override def before(): Unit = {
+    super.before()
+    val setting = EnvironmentSettings.newInstance().useBlinkPlanner().inStreamingMode().build()
+    val config = new TestTableConfig
+    this.tEnv = StreamTableEnvironmentImpl.create(env, setting, config)
+  }
+
   @Test
   def testProcTimeBoundedRowsOver(): Unit = {
 
@@ -59,8 +68,8 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
       """.stripMargin
     val t1 = tEnv.sqlQuery(sql)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.seconds(2))
-    val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate")
+    val queryConfig = new TestStreamQueryConfig(Time.seconds(2), Time.seconds(4))
+    val testHarness = createHarnessTester(t1.toAppendStream[Row](queryConfig), "OverAggregate")
     val assertor = new BaseRowHarnessAssertor(
       Array(Types.LONG, Types.STRING, Types.LONG, Types.LONG, Types.LONG, Types.LONG, Types.LONG))
 
@@ -168,8 +177,8 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
       """.stripMargin
     val t1 = tEnv.sqlQuery(sql)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.seconds(2))
-    val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate")
+    val queryConfig = new TestStreamQueryConfig(Time.seconds(2), Time.seconds(4))
+    val testHarness = createHarnessTester(t1.toAppendStream[Row](queryConfig), "OverAggregate")
     val assertor = new BaseRowHarnessAssertor(
       Array(Types.LONG, Types.STRING, Types.LONG, Types.LONG, Types.LONG, Types.LONG))
     testHarness.open()
@@ -297,8 +306,8 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
       """.stripMargin
     val t1 = tEnv.sqlQuery(sql)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.seconds(2))
-    val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate")
+    val queryConfig = new TestStreamQueryConfig(Time.seconds(2), Time.seconds(4))
+    val testHarness = createHarnessTester(t1.toAppendStream[Row](queryConfig), "OverAggregate")
     val assertor = new BaseRowHarnessAssertor(
       Array(Types.LONG, Types.STRING, Types.LONG, Types.LONG, Types.LONG, Types.LONG))
 
@@ -400,8 +409,8 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
       """.stripMargin
     val t1 = tEnv.sqlQuery(sql)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.seconds(1))
-    val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate")
+    val queryConfig = new TestStreamQueryConfig(Time.seconds(1), Time.seconds(2))
+    val testHarness = createHarnessTester(t1.toAppendStream[Row](queryConfig), "OverAggregate")
     val assertor = new BaseRowHarnessAssertor(
       Array(Types.LONG, Types.STRING, Types.LONG, Types.LONG, Types.LONG))
 
@@ -549,8 +558,8 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
       """.stripMargin
     val t1 = tEnv.sqlQuery(sql)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.seconds(1))
-    val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate")
+    val queryConfig = new TestStreamQueryConfig(Time.seconds(1), Time.seconds(2))
+    val testHarness = createHarnessTester(t1.toAppendStream[Row](queryConfig), "OverAggregate")
     val assertor = new BaseRowHarnessAssertor(
       Array(Types.LONG, Types.STRING, Types.LONG, Types.LONG, Types.LONG))
 
@@ -697,8 +706,8 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
       """.stripMargin
     val t1 = tEnv.sqlQuery(sql)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.seconds(1))
-    val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate")
+    val queryConfig = new TestStreamQueryConfig(Time.seconds(1), Time.seconds(2))
+    val testHarness = createHarnessTester(t1.toAppendStream[Row](queryConfig), "OverAggregate")
     val assertor = new BaseRowHarnessAssertor(
       Array(Types.LONG, Types.STRING, Types.LONG, Types.LONG, Types.LONG))
 
@@ -835,8 +844,8 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
       """.stripMargin
     val t1 = tEnv.sqlQuery(sql)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.seconds(1))
-    val testHarness = createHarnessTester(t1.toAppendStream[Row], "OverAggregate")
+    val queryConfig = new TestStreamQueryConfig(Time.seconds(1), Time.seconds(2))
+    val testHarness = createHarnessTester(t1.toAppendStream[Row](queryConfig), "OverAggregate")
     val assertor = new BaseRowHarnessAssertor(
       Array(Types.LONG, Types.STRING, Types.LONG, Types.LONG, Types.LONG))
 
@@ -950,4 +959,28 @@ class OverWindowHarnessTest(mode: StateBackendMode) extends HarnessTestBase(mode
     assertor.assertOutputEqualsSorted("result mismatch", expectedOutput, result)
     testHarness.close()
   }
+
+  /**
+    * Test class used to test min and max retention time.
+    */
+  class TestStreamQueryConfig(min: Time, max: Time) extends StreamQueryConfig {
+    override def getMinIdleStateRetentionTime: Long = min.toMilliseconds
+    override def getMaxIdleStateRetentionTime: Long = max.toMilliseconds
+  }
+
+  class TestTableConfig extends TableConfig {
+
+    private var minIdleStateRetentionTime = 0L
+
+    private var maxIdleStateRetentionTime = 0L
+
+    override def getMinIdleStateRetentionTime: Long = minIdleStateRetentionTime
+
+    override def getMaxIdleStateRetentionTime: Long = maxIdleStateRetentionTime
+
+    override def setIdleStateRetentionTime(minTime: Time, maxTime: Time): Unit = {
+      minIdleStateRetentionTime = minTime.toMilliseconds
+      maxIdleStateRetentionTime = maxTime.toMilliseconds
+    }
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala
index bcb7639..0cde6e1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/OverWindowITCase.scala
@@ -18,7 +18,6 @@
 
 package org.apache.flink.table.runtime.stream.sql
 
-import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
@@ -256,8 +255,6 @@ class OverWindowITCase(mode: StateBackendMode) extends StreamingWithStateTestBas
 
   @Test
   def testProcTimeUnboundedNonPartitionedRangeOver(): Unit = {
-    tEnv.getConfig.withIdleStateRetentionTime(Time.hours(2))
-
     val t1 = failingDataSource(data).toTable(tEnv, 'a, 'b, 'c, 'proctime.proctime)
 
     tEnv.registerTable("T1", t1)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala
index 1bccf94..fd49fde 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SortITCase.scala
@@ -66,7 +66,8 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     tEnv.registerTable("a", da)
 
     val sink = new TestingRetractSink
-    tEnv.getConfig.getConf.setBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+    tEnv.getConfig.getConfiguration.setBoolean(
+      StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
     val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
     results.addSink(sink).setParallelism(1)
     env.execute()
@@ -94,7 +95,8 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     tEnv.registerTable("a", da)
 
     val sink = new TestingRetractSink
-    tEnv.getConfig.getConf.setBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+    tEnv.getConfig.getConfiguration.setBoolean(
+      StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
     val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
     results.addSink(sink).setParallelism(1)
     env.execute()
@@ -122,7 +124,8 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     tEnv.registerTable("a", da)
 
     val sink = new TestingRetractSink
-    tEnv.getConfig.getConf.setBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+    tEnv.getConfig.getConfiguration.setBoolean(
+      StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
     val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
     results.addSink(sink).setParallelism(1)
     env.execute()
@@ -150,7 +153,8 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     tEnv.registerTable("a", da)
 
     val sink = new TestingRetractSink
-    tEnv.getConfig.getConf.setBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+    tEnv.getConfig.getConfiguration.setBoolean(
+      StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
     val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
     results.addSink(sink).setParallelism(1)
     env.execute()
@@ -184,7 +188,8 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     tEnv.registerTable("a", da)
 
     val sink = new TestingRetractSink
-    tEnv.getConfig.getConf.setBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+    tEnv.getConfig.getConfiguration.setBoolean(
+      StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
     val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
     results.addSink(sink).setParallelism(1)
     env.execute()
@@ -218,7 +223,8 @@ class SortITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     tEnv.registerTable("a", da)
 
     val sink = new TestingRetractSink
-    tEnv.getConfig.getConf.setBoolean(StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
+    tEnv.getConfig.getConfiguration.setBoolean(
+      StreamExecSort.SQL_EXEC_SORT_NON_TEMPORAL_ENABLED, true)
     val results = tEnv.sqlQuery(sqlQuery).toRetractStream[Row]
     results.addSink(sink).setParallelism(1)
     env.execute()
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala
index 868a517..750086e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/SplitAggregateITCase.scala
@@ -50,10 +50,10 @@ class SplitAggregateITCase(
     super.before()
 
     if (partialAggMode.isPartialAggEnabled) {
-      tEnv.getConfig.getConf.setBoolean(
+      tEnv.getConfig.getConfiguration.setBoolean(
         OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, true)
     } else {
-      tEnv.getConfig.getConf.setBoolean(
+      tEnv.getConfig.getConfiguration.setBoolean(
         OptimizerConfigOptions.SQL_OPTIMIZER_DISTINCT_AGG_SPLIT_ENABLED, false)
     }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/WindowAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/WindowAggregateITCase.scala
index e2b782e..9ffb779 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/WindowAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/sql/WindowAggregateITCase.scala
@@ -22,13 +22,15 @@ package org.apache.flink.table.runtime.stream.sql
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.Types
+import org.apache.flink.table.api.{TableConfig, Types}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.plan.util.JavaUserDefinedAggFunctions.{ConcatDistinctAggFunction, WeightedAvg}
+import org.apache.flink.table.plan.util.WindowEmitStrategy.{SQL_EXEC_EMIT_LATE_FIRE_DELAY, SQL_EXEC_EMIT_LATE_FIRE_ENABLED}
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.StateBackendMode
 import org.apache.flink.table.runtime.utils.TimeTestUtil.TimestampAndWatermarkWithOffset
 import org.apache.flink.table.runtime.utils._
 import org.apache.flink.types.Row
+import org.apache.flink.table.util.TableConfigUtils.getMillisecondFromConfigDuration
 
 import org.junit.Assert.assertEquals
 import org.junit.Test
@@ -146,9 +148,10 @@ class WindowAggregateITCase(mode: StateBackendMode)
   @Test
   def testEventTimeTumblingWindowWithAllowLateness(): Unit = {
     // wait 10 millisecond for late elements
-    tEnv.getConfig.withIdleStateRetentionTime(Time.milliseconds(10))
+    tEnv.getConfig.setIdleStateRetentionTime(
+      Time.milliseconds(10), Time.minutes(6))
     // emit result without delay after watermark
-    tEnv.getConfig.withLateFireInterval(Time.of(0, TimeUnit.NANOSECONDS))
+    withLateFireDelay(tEnv.getConfig, Time.of(0, TimeUnit.NANOSECONDS))
     val data = List(
       (1L, 1, "Hi"),
       (2L, 2, "Hello"),
@@ -247,4 +250,17 @@ class WindowAggregateITCase(mode: StateBackendMode)
     )
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
   }
+
+  private def withLateFireDelay(tableConfig: TableConfig, interval: Time): Unit = {
+    val intervalInMillis = interval.toMilliseconds
+    val preLateFireInterval = getMillisecondFromConfigDuration(tableConfig,
+      SQL_EXEC_EMIT_LATE_FIRE_DELAY)
+    if (preLateFireInterval != null && (preLateFireInterval != intervalInMillis)) {
+      // lateFireInterval of the two query config is not equal and not the default
+      throw new RuntimeException(
+        "Currently not support different lateFireInterval configs in one job")
+    }
+    tableConfig.getConfiguration.setBoolean(SQL_EXEC_EMIT_LATE_FIRE_ENABLED, true)
+    tableConfig.getConfiguration.setString(SQL_EXEC_EMIT_LATE_FIRE_DELAY, intervalInMillis + " ms")
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala
index 7b590d0..7207ff9 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/AggregateITCase.scala
@@ -46,7 +46,7 @@ class AggregateITCase(mode: StateBackendMode) extends StreamingWithStateTestBase
   @Before
   override def before(): Unit = {
     super.before()
-    tEnv.getConfig.withIdleStateRetentionTime(Time.hours(1))
+    tEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala
index aed2941..07373e3 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/stream/table/JoinITCase.scala
@@ -81,7 +81,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
   @Before
   override def before(): Unit = {
     super.before()
-    tEnv.getConfig.withIdleStateRetentionTime(Time.hours(1))
+    tEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
   }
 
   @Test
@@ -1055,7 +1055,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       .groupBy('bb)
       .select('bb, 'c.count as 'c)
 
-    tEnv.getConfig.withIdleStateRetentionTime(Time.hours(1))
+    tEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
 
     val t = leftTableWithPk
       .join(rightTableWithPk, 'b === 'bb)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
index db92495..fd2e489 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/BatchTestBase.scala
@@ -66,12 +66,12 @@ class BatchTestBase extends BatchAbstractTestBase {
 
   @Before
   def before(): Unit = {
-    conf.getConf.setInteger(
+    conf.getConfiguration.setInteger(
       ExecutionConfigOptions.SQL_RESOURCE_DEFAULT_PARALLELISM, DEFAULT_PARALLELISM)
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM, 2)
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
-    conf.getConf.setInteger(ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM, 1)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_AGG_TABLE_MEM, 2)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_HASH_JOIN_TABLE_MEM, 2)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_SORT_BUFFER_MEM, 1)
+    conf.getConfiguration.setInteger(ExecutionConfigOptions.SQL_RESOURCE_EXTERNAL_BUFFER_MEM, 1)
   }
 
   /**
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala
index 0e3970e..0c49792 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithAggTestBase.scala
@@ -19,10 +19,12 @@ package org.apache.flink.table.runtime.utils
 
 import java.util
 import org.apache.flink.api.common.time.Time
-import org.apache.flink.table.api.{AggPhaseEnforcer, OptimizerConfigOptions}
+import org.apache.flink.table.api.OptimizerConfigOptions
 import org.apache.flink.table.runtime.utils.StreamingWithAggTestBase._
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode}
 import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.{MiniBatchMode, MiniBatchOff, MiniBatchOn}
+import org.apache.flink.table.util.AggregatePhaseStrategy
+
 import org.junit.Before
 import org.junit.runners.Parameterized
 
@@ -37,15 +39,15 @@ class StreamingWithAggTestBase(
   override def before(): Unit = {
     super.before()
     // in order to cover more code paths
-    tEnv.getConfig.withIdleStateRetentionTime(Time.hours(1))
+    tEnv.getConfig.setIdleStateRetentionTime(Time.hours(1), Time.hours(2))
     if (aggMode.isLocalAggEnabled) {
-      tEnv.getConfig.getConf.setString(
+      tEnv.getConfig.getConfiguration.setString(
         OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
-        AggPhaseEnforcer.TWO_PHASE.toString)
+        AggregatePhaseStrategy.TWO_PHASE.toString)
     } else {
-      tEnv.getConfig.getConf.setString(
+      tEnv.getConfig.getConfiguration.setString(
         OptimizerConfigOptions.SQL_OPTIMIZER_AGG_PHASE_STRATEGY,
-        AggPhaseEnforcer.ONE_PHASE.toString)
+        AggregatePhaseStrategy.ONE_PHASE.toString)
     }
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala
index 397427b..0afccfe 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/runtime/utils/StreamingWithMiniBatchTestBase.scala
@@ -17,7 +17,7 @@
  */
 package org.apache.flink.table.runtime.utils
 
-import org.apache.flink.table.api.ExecutionConfigOptions
+import org.apache.flink.table.api.ExecutionConfigOptions.{SQL_EXEC_MINIBATCH_ALLOW_LATENCY, SQL_EXEC_MINIBATCH_SIZE, SQL_EXEC_MINIBATCH_ENABLED}
 import org.apache.flink.table.runtime.utils.StreamingWithStateTestBase.{HEAP_BACKEND, ROCKSDB_BACKEND, StateBackendMode}
 import org.apache.flink.table.runtime.utils.StreamingWithMiniBatchTestBase.{MiniBatchMode, MiniBatchOff, MiniBatchOn}
 
@@ -37,13 +37,11 @@ abstract class StreamingWithMiniBatchTestBase(
     val tableConfig = tEnv.getConfig
     miniBatch match {
       case MiniBatchOn =>
-        tableConfig.getConf.setBoolean(
-          ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-        tableConfig.getConf.setString(
-          ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
-        tableConfig.getConf.setLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 3L)
+        tableConfig.getConfiguration.setBoolean(SQL_EXEC_MINIBATCH_ENABLED, true)
+        tableConfig.getConfiguration.setString(SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
+        tableConfig.getConfiguration.setLong(SQL_EXEC_MINIBATCH_SIZE, 3L)
       case MiniBatchOff =>
-        tableConfig.getConf.removeConfig(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY)
+        tableConfig.getConfiguration.removeConfig(SQL_EXEC_MINIBATCH_ALLOW_LATENCY)
     }
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
index 9ff27ea..a59622f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/util/TableTestBase.scala
@@ -41,17 +41,15 @@ import org.apache.flink.table.functions.{AggregateFunction, ScalarFunction, Tabl
 import org.apache.flink.table.operations.{CatalogSinkModifyOperation, DataStreamQueryOperation, ModifyOperation, PlannerQueryOperation, QueryOperation, RichTableSourceQueryOperation}
 import org.apache.flink.table.plan.nodes.calcite.LogicalWatermarkAssigner
 import org.apache.flink.table.plan.nodes.exec.ExecNode
-import org.apache.flink.table.plan.optimize.program.{FlinkBatchProgram, FlinkStreamProgram}
+import org.apache.flink.table.plan.optimize.program.{BatchOptimizeContext, FlinkBatchProgram, FlinkChainedProgram, FlinkStreamProgram, StreamOptimizeContext}
 import org.apache.flink.table.plan.stats.FlinkStatistic
 import org.apache.flink.table.plan.util.{ExecNodePlanDumper, FlinkRelOptUtil}
 import org.apache.flink.table.planner.PlannerBase
 import org.apache.flink.table.runtime.utils.{TestingAppendTableSink, TestingRetractTableSink, TestingUpsertTableSink}
 import org.apache.flink.table.sinks._
 import org.apache.flink.table.sources.{StreamTableSource, TableSource}
-import org.apache.flink.table.types.TypeInfoLogicalTypeConverter
 import org.apache.flink.table.types.TypeInfoLogicalTypeConverter.fromLogicalTypeToTypeInfo
 import org.apache.flink.table.types.logical.LogicalType
-import org.apache.flink.table.typeutils.BaseRowTypeInfo
 import org.apache.flink.types.Row
 
 import org.apache.calcite.rel.RelNode
@@ -698,7 +696,7 @@ case class StreamTableTestUtil(
   }
 
   def buildStreamProgram(firstProgramNameToRemove: String): Unit = {
-    val program = FlinkStreamProgram.buildProgram(tableEnv.getConfig.getConf)
+    val program = FlinkStreamProgram.buildProgram(tableEnv.getConfig.getConfiguration)
     var startRemove = false
     program.getProgramNames.foreach {
       name =>
@@ -709,17 +707,23 @@ case class StreamTableTestUtil(
           program.remove(name)
         }
     }
-    val calciteConfig = CalciteConfig.createBuilder(tableEnv.getConfig.getCalciteConfig)
+    replaceStreamProgram(program)
+  }
+
+  def replaceStreamProgram(program: FlinkChainedProgram[StreamOptimizeContext]): Unit = {
+    var calciteConfig = TableConfigUtils.getCalciteConfig(tableEnv.getConfig)
+    calciteConfig = CalciteConfig.createBuilder(calciteConfig)
       .replaceStreamProgram(program).build()
-    tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    tableEnv.getConfig.setPlannerConfig(calciteConfig)
   }
 
+
   def enableMiniBatch(): Unit = {
-    tableEnv.getConfig.getConf.setBoolean(
+    tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ENABLED, true)
-    tableEnv.getConfig.getConf.setString(
+    tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.SQL_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
-    tableEnv.getConfig.getConf.setLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 3L)
+    tableEnv.getConfig.getConfiguration.setLong(ExecutionConfigOptions.SQL_EXEC_MINIBATCH_SIZE, 3L)
   }
 
   def createAppendTableSink(
@@ -770,7 +774,7 @@ case class BatchTableTestUtil(
   extends TableTestUtil(test, true, catalogManager) {
 
   def buildBatchProgram(firstProgramNameToRemove: String): Unit = {
-    val program = FlinkBatchProgram.buildProgram(tableEnv.getConfig.getConf)
+    val program = FlinkBatchProgram.buildProgram(tableEnv.getConfig.getConfiguration)
     var startRemove = false
     program.getProgramNames.foreach {
       name =>
@@ -781,9 +785,21 @@ case class BatchTableTestUtil(
           program.remove(name)
         }
     }
-    val calciteConfig = CalciteConfig.createBuilder(tableEnv.getConfig.getCalciteConfig)
+    replaceBatchProgram(program)
+  }
+
+  def replaceBatchProgram(program: FlinkChainedProgram[BatchOptimizeContext]): Unit = {
+    var calciteConfig = TableConfigUtils.getCalciteConfig(tableEnv.getConfig)
+    calciteConfig = CalciteConfig.createBuilder(calciteConfig)
       .replaceBatchProgram(program).build()
-    tableEnv.getConfig.setCalciteConfig(calciteConfig)
+    tableEnv.getConfig.setPlannerConfig(calciteConfig)
+  }
+
+  def getBatchProgram(): FlinkChainedProgram[BatchOptimizeContext] = {
+    val tableConfig = tableEnv.getConfig
+    val calciteConfig = TableConfigUtils.getCalciteConfig(tableConfig)
+    calciteConfig.getBatchProgram.getOrElse(FlinkBatchProgram.buildProgram(
+      tableConfig.getConfiguration))
   }
 
   def createCollectTableSink(