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 2020/05/13 01:38:57 UTC

[flink] branch master updated: [FLINK-16364][table] Deprecate the methods in TableEnvironment & Table proposed by FLIP-84

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 5324b0c  [FLINK-16364][table] Deprecate the methods in TableEnvironment & Table proposed by FLIP-84
5324b0c is described below

commit 5324b0c53d3ad73cea1f97e9861b7b7373d49f77
Author: godfrey he <go...@163.com>
AuthorDate: Wed May 13 09:38:16 2020 +0800

    [FLINK-16364][table] Deprecate the methods in TableEnvironment & Table proposed by FLIP-84
    
    which includes:
    TableEnvironment.sqlUpdate(String)
    TableEnvironment.insertInto(String, Table)
    TableEnvironment.execute(String)
    TableEnvironment.explain(boolean)
    TableEnvironment.fromTableSource(TableSource<?>)
    Table.insertInto(String)
    
    Please use TableEnvironment#executeSql for single statement,
    use Table#executeInsert for single sink,
    use TableEnvironment#createStatementSet for multiple sinks.
    
    This closes #11297
---
 .../cassandra/CassandraConnectorITCase.java        |   5 +-
 .../flink/connectors/hive/HiveDialectTest.java     |  14 +-
 .../flink/connectors/hive/HiveTableSinkTest.java   |  13 +-
 .../flink/connectors/hive/HiveTableSourceTest.java |  18 +-
 .../connectors/hive/TableEnvHiveConnectorTest.java |  63 ++--
 .../table/catalog/hive/HiveCatalogITCase.java      |  22 +-
 .../catalog/hive/HiveCatalogUseBlinkITCase.java    |   5 +-
 .../connectors/kafka/KafkaTableTestBase.java       |   5 +-
 .../flink/addons/hbase/HBaseConnectorITCase.java   |  15 +-
 .../flink/api/java/io/jdbc/JDBCDataTypeTest.java   |   2 +-
 .../api/java/io/jdbc/JDBCTableSourceITCase.java    |   4 +-
 .../java/io/jdbc/JDBCUpsertTableSinkITCase.java    |  30 +-
 .../io/jdbc/catalog/PostgresCatalogITCase.java     |   6 +-
 .../flink/sql/tests/BatchSQLTestProgram.java       |   6 +-
 .../apache/flink/table/tpcds/TpcdsTestProgram.java |   8 +-
 .../examples/java/StreamWindowSQLExample.java      |   2 +-
 flink-python/pyflink/table/table.py                |   9 +
 flink-python/pyflink/table/table_environment.py    |  18 ++
 flink-python/pyflink/table/tests/test_sql.py       |   2 +-
 .../table/tests/test_table_environment_api.py      |  25 +-
 .../client/python/PythonFunctionFactoryTest.java   |  18 +-
 .../client/gateway/local/ExecutionContext.java     |   2 +-
 .../table/client/gateway/local/LocalExecutor.java  |  14 +-
 .../client/gateway/local/ExecutionContextTest.java |   2 +-
 .../java/org/apache/flink/table/api/Table.java     |   3 +
 .../apache/flink/table/api/TableEnvironment.java   |  20 +-
 .../table/api/internal/TableEnvironmentImpl.java   |  22 +-
 .../table/planner/sinks/BatchSelectTableSink.java  |   3 +-
 .../sinks/SelectTableSinkSchemaConverter.java      |  28 +-
 .../table/planner/sinks/StreamSelectTableSink.java |   3 +-
 .../flink/table/planner/catalog/CatalogITCase.java |   2 +-
 .../planner/runtime/stream/sql/FunctionITCase.java | 175 ++++++------
 .../runtime/stream/table/FunctionITCase.java       |  31 +-
 .../planner/runtime/stream/table/ValuesITCase.java |  18 +-
 .../flink/table/api/TableEnvironmentITCase.scala   |  53 +++-
 .../flink/table/api/TableEnvironmentTest.scala     |   8 +-
 .../apache/flink/table/api/batch/ExplainTest.scala |  35 ++-
 .../flink/table/api/stream/ExplainTest.scala       |  48 ++--
 .../table/planner/catalog/CatalogTableITCase.scala | 183 ++++++------
 .../table/planner/catalog/CatalogTableTest.scala   |   2 +-
 .../table/planner/catalog/CatalogViewITCase.scala  |  22 +-
 .../expressions/utils/ExpressionTestBase.scala     |   2 +-
 .../plan/batch/sql/DagOptimizationTest.scala       | 137 +++++----
 .../plan/batch/sql/PartitionableSinkTest.scala     |  16 +-
 .../table/planner/plan/batch/sql/SinkTest.scala    |  12 +-
 .../plan/batch/sql/agg/WindowAggregateTest.scala   |   2 +-
 .../planner/plan/common/TableFactoryTest.scala     |   8 +-
 .../table/planner/plan/hint/OptionsHintTest.scala  |   9 +-
 ...shFilterIntoLegacyTableSourceScanRuleTest.scala |   2 +-
 ...hProjectIntoLegacyTableSourceScanRuleTest.scala |   4 +-
 .../stream/ChangelogModeInferenceTest.scala        |  12 +-
 .../plan/stream/sql/DagOptimizationTest.scala      | 166 ++++++-----
 .../stream/sql/MiniBatchIntervalInferTest.scala    |  44 +--
 .../plan/stream/sql/ModifiedMonotonicityTest.scala |  13 +-
 .../plan/stream/sql/PartitionableSinkTest.scala    |  14 +-
 .../table/planner/plan/stream/sql/RankTest.scala   |  28 +-
 .../table/planner/plan/stream/sql/SinkTest.scala   |  62 ++--
 .../planner/plan/stream/sql/SortLimitTest.scala    |   8 +-
 .../planner/plan/stream/sql/TableScanTest.scala    |  18 +-
 .../plan/stream/sql/agg/AggregateTest.scala        |  14 +-
 .../stream/sql/agg/DistinctAggregateTest.scala     |   7 +-
 .../plan/stream/sql/agg/WindowAggregateTest.scala  |   8 +-
 .../planner/plan/stream/sql/join/JoinTest.scala    |  58 ++--
 .../plan/stream/sql/join/LookupJoinTest.scala      |   2 +-
 .../plan/stream/table/TableSourceTest.scala        |  24 +-
 .../table/validation/TableSinkValidationTest.scala |   7 +-
 .../planner/runtime/FileSystemITCaseBase.scala     |  28 +-
 .../planner/runtime/batch/sql/LimitITCase.scala    |   2 +-
 .../batch/sql/PartitionableSinkITCase.scala        |  20 +-
 .../runtime/batch/sql/join/JoinITCase.scala        |  24 +-
 .../runtime/batch/table/OverWindowITCase.scala     |  50 ++--
 .../runtime/batch/table/TableSinkITCase.scala      |  35 +--
 .../runtime/stream/FsStreamingSinkITCaseBase.scala |  10 +-
 .../runtime/stream/sql/AggregateITCase.scala       |   3 +-
 .../planner/runtime/stream/sql/CalcITCase.scala    |   3 +-
 .../runtime/stream/sql/CorrelateITCase.scala       |  21 +-
 .../runtime/stream/sql/Limit0RemoveITCase.scala    |  24 +-
 .../planner/runtime/stream/sql/RankITCase.scala    |  57 ++--
 .../runtime/stream/sql/TemporalJoinITCase.scala    |   4 +-
 .../runtime/stream/sql/TimeAttributeITCase.scala   |   8 +-
 .../planner/runtime/stream/sql/UnnestITCase.scala  |   3 +-
 .../runtime/stream/sql/WindowAggregateITCase.scala |   3 +-
 .../runtime/stream/table/AggregateITCase.scala     |   3 +-
 .../planner/runtime/stream/table/JoinITCase.scala  |  18 +-
 .../runtime/stream/table/TableSinkITCase.scala     |  94 +++---
 .../planner/runtime/utils/BatchTableEnvUtil.scala  |   5 +-
 .../planner/runtime/utils/BatchTestBase.scala      |  11 +-
 .../planner/runtime/utils/StreamingTestBase.scala  |  11 +
 .../table/planner/runtime/utils/TableEnvUtil.scala |  41 +++
 .../flink/table/planner/utils/TableTestBase.scala  | 317 +++++++++------------
 .../flink/table/sinks/BatchSelectTableSink.java    |   3 +-
 .../sinks/SelectTableSinkSchemaConverter.java      |   4 +-
 .../flink/table/sinks/StreamSelectTableSink.java   |   2 +-
 .../flink/table/api/internal/TableEnvImpl.scala    |  22 +-
 .../runtime/batch/sql/TableEnvironmentITCase.scala |  31 ++
 95 files changed, 1342 insertions(+), 1156 deletions(-)

diff --git a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
index 28843f4..6a16d10 100644
--- a/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
+++ b/flink-connectors/flink-connector-cassandra/src/test/java/org/apache/flink/streaming/connectors/cassandra/CassandraConnectorITCase.java
@@ -44,6 +44,7 @@ import org.apache.flink.streaming.api.datastream.DataStreamSource;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.functions.sink.SinkContextUtil;
 import org.apache.flink.streaming.runtime.operators.WriteAheadSinkTestBase;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.testutils.junit.FailsOnJava11;
 import org.apache.flink.types.Row;
@@ -469,9 +470,9 @@ public class CassandraConnectorITCase extends WriteAheadSinkTestBase<Tuple3<Stri
 				new TypeInformation[]{Types.STRING, Types.INT, Types.INT}
 			));
 
-		tEnv.sqlQuery("select * from testFlinkTable").insertInto("cassandraTable");
+		TableResult tableResult = tEnv.sqlQuery("select * from testFlinkTable").executeInsert("cassandraTable");
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 
-		tEnv.execute("job name");
 		ResultSet rs = session.execute(injectTableName(SELECT_DATA_QUERY));
 
 		// validate that all input was correctly written to Cassandra
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectTest.java
index f81c068..9f65c46 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveDialectTest.java
@@ -70,13 +70,13 @@ public class HiveDialectTest {
 
 	@Test
 	public void testCreateDatabase() throws Exception {
-		tableEnv.sqlUpdate("create database db1 comment 'db1 comment'");
+		tableEnv.executeSql("create database db1 comment 'db1 comment'");
 		Database db = hiveCatalog.getHiveDatabase("db1");
 		assertEquals("db1 comment", db.getDescription());
 		assertFalse(Boolean.parseBoolean(db.getParameters().get(CatalogConfig.IS_GENERIC)));
 
 		String db2Location = warehouse + "/db2_location";
-		tableEnv.sqlUpdate(String.format("create database db2 location '%s' with dbproperties('k1'='v1')", db2Location));
+		tableEnv.executeSql(String.format("create database db2 location '%s' with dbproperties('k1'='v1')", db2Location));
 		db = hiveCatalog.getHiveDatabase("db2");
 		assertEquals(db2Location, new URI(db.getLocationUri()).getPath());
 		assertEquals("v1", db.getParameters().get("k1"));
@@ -85,8 +85,8 @@ public class HiveDialectTest {
 	@Test
 	public void testAlterDatabase() throws Exception {
 		// alter properties
-		tableEnv.sqlUpdate("create database db1 with dbproperties('k1'='v1')");
-		tableEnv.sqlUpdate("alter database db1 set dbproperties ('k1'='v11','k2'='v2')");
+		tableEnv.executeSql("create database db1 with dbproperties('k1'='v1')");
+		tableEnv.executeSql("alter database db1 set dbproperties ('k1'='v11','k2'='v2')");
 		Database db = hiveCatalog.getHiveDatabase("db1");
 		// there's an extra is_generic property
 		assertEquals(3, db.getParametersSize());
@@ -94,12 +94,12 @@ public class HiveDialectTest {
 		assertEquals("v2", db.getParameters().get("k2"));
 
 		// alter owner
-		tableEnv.sqlUpdate("alter database db1 set owner user user1");
+		tableEnv.executeSql("alter database db1 set owner user user1");
 		db = hiveCatalog.getHiveDatabase("db1");
 		assertEquals("user1", db.getOwnerName());
 		assertEquals(PrincipalType.USER, db.getOwnerType());
 
-		tableEnv.sqlUpdate("alter database db1 set owner role role1");
+		tableEnv.executeSql("alter database db1 set owner role role1");
 		db = hiveCatalog.getHiveDatabase("db1");
 		assertEquals("role1", db.getOwnerName());
 		assertEquals(PrincipalType.ROLE, db.getOwnerType());
@@ -107,7 +107,7 @@ public class HiveDialectTest {
 		// alter location
 		if (hiveCatalog.getHiveVersion().compareTo("2.4.0") >= 0) {
 			String newLocation = warehouse + "/db1_new_location";
-			tableEnv.sqlUpdate(String.format("alter database db1 set location '%s'", newLocation));
+			tableEnv.executeSql(String.format("alter database db1 set location '%s'", newLocation));
 			db = hiveCatalog.getHiveDatabase("db1");
 			assertEquals(newLocation, new URI(db.getLocationUri()).getPath());
 		}
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java
index 718a499..2b0d127 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSinkTest.java
@@ -33,6 +33,7 @@ import org.apache.flink.table.catalog.ObjectPath;
 import org.apache.flink.table.catalog.config.CatalogConfig;
 import org.apache.flink.table.catalog.hive.HiveCatalog;
 import org.apache.flink.table.catalog.hive.HiveTestUtils;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
 import org.apache.flink.table.sources.InputFormatTableSource;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.table.types.utils.TypeConversions;
@@ -97,8 +98,7 @@ public class HiveTableSinkTest {
 		tableEnv.registerTable("src", src);
 
 		tableEnv.registerCatalog("hive", hiveCatalog);
-		tableEnv.sqlQuery("select * from src").insertInto("hive.`default`.dest");
-		tableEnv.execute("mytest");
+		TableEnvUtil.execInsertTableAndWaitResult(tableEnv.sqlQuery("select * from src"), "hive.`default`.dest");
 
 		verifyWrittenData(toWrite, hiveShell.executeQuery("select * from " + tblName));
 
@@ -139,8 +139,7 @@ public class HiveTableSinkTest {
 		tableEnv.registerTable("complexSrc", src);
 
 		tableEnv.registerCatalog("hive", hiveCatalog);
-		tableEnv.sqlQuery("select * from complexSrc").insertInto("hive.`default`.dest");
-		tableEnv.execute("mytest");
+		TableEnvUtil.execInsertTableAndWaitResult(tableEnv.sqlQuery("select * from complexSrc"), "hive.`default`.dest");
 
 		List<String> result = hiveShell.executeQuery("select * from " + tblName);
 		assertEquals(1, result.size());
@@ -178,8 +177,7 @@ public class HiveTableSinkTest {
 		Table src = tableEnv.fromTableSource(new CollectionTableSource(toWrite, rowTypeInfo));
 		tableEnv.registerTable("nestedSrc", src);
 		tableEnv.registerCatalog("hive", hiveCatalog);
-		tableEnv.sqlQuery("select * from nestedSrc").insertInto("hive.`default`.dest");
-		tableEnv.execute("mytest");
+		TableEnvUtil.execInsertTableAndWaitResult(tableEnv.sqlQuery("select * from nestedSrc"), "hive.`default`.dest");
 
 		List<String> result = hiveShell.executeQuery("select * from " + tblName);
 		assertEquals(1, result.size());
@@ -203,8 +201,7 @@ public class HiveTableSinkTest {
 			tableEnv.registerCatalog(hiveCatalog.getName(), hiveCatalog);
 			tableEnv.useCatalog(hiveCatalog.getName());
 
-			tableEnv.sqlUpdate("insert into db1.dest select * from db1.src");
-			tableEnv.execute("write to dest");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest select * from db1.src");
 			List<String> results = hiveShell.executeQuery("select * from db1.dest");
 			assertEquals(1, results.size());
 			String[] cols = results.get(0).split("\t");
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java
index 6d9938e..50877b2 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/HiveTableSourceTest.java
@@ -216,7 +216,7 @@ public class HiveTableSourceTest {
 		tEnv.registerCatalog(catalogName, hiveCatalog);
 		Table src = tEnv.sqlQuery("select * from hive.source_db.test_table_pt_1 where pt = 0");
 		// first check execution plan to ensure partition prunning works
-		String[] explain = tEnv.explain(src).split("==.*==\n");
+		String[] explain = src.explain().split("==.*==\n");
 		assertEquals(4, explain.length);
 		String optimizedLogicalPlan = explain[2];
 		String physicalExecutionPlan = explain[3];
@@ -251,7 +251,7 @@ public class HiveTableSourceTest {
 			tableEnv.registerCatalog(catalog.getName(), catalog);
 			tableEnv.useCatalog(catalog.getName());
 			Table query = tableEnv.sqlQuery("select x from db1.part where p1>1 or p2<>'a' order by x");
-			String[] explain = tableEnv.explain(query).split("==.*==\n");
+			String[] explain = query.explain().split("==.*==\n");
 			assertFalse(catalog.fallback);
 			String optimizedPlan = explain[2];
 			assertTrue(optimizedPlan, optimizedPlan.contains("PartitionPruned: true, PartitionNums: 3"));
@@ -259,7 +259,7 @@ public class HiveTableSourceTest {
 			assertEquals("[2, 3, 4]", results.toString());
 
 			query = tableEnv.sqlQuery("select x from db1.part where p1>2 and p2<='a' order by x");
-			explain = tableEnv.explain(query).split("==.*==\n");
+			explain = query.explain().split("==.*==\n");
 			assertFalse(catalog.fallback);
 			optimizedPlan = explain[2];
 			assertTrue(optimizedPlan, optimizedPlan.contains("PartitionPruned: true, PartitionNums: 0"));
@@ -267,7 +267,7 @@ public class HiveTableSourceTest {
 			assertEquals("[]", results.toString());
 
 			query = tableEnv.sqlQuery("select x from db1.part where p1 in (1,3,5) order by x");
-			explain = tableEnv.explain(query).split("==.*==\n");
+			explain = query.explain().split("==.*==\n");
 			assertFalse(catalog.fallback);
 			optimizedPlan = explain[2];
 			assertTrue(optimizedPlan, optimizedPlan.contains("PartitionPruned: true, PartitionNums: 2"));
@@ -275,7 +275,7 @@ public class HiveTableSourceTest {
 			assertEquals("[1, 3]", results.toString());
 
 			query = tableEnv.sqlQuery("select x from db1.part where (p1=1 and p2='a') or ((p1=2 and p2='b') or p2='d') order by x");
-			explain = tableEnv.explain(query).split("==.*==\n");
+			explain = query.explain().split("==.*==\n");
 			assertFalse(catalog.fallback);
 			optimizedPlan = explain[2];
 			assertTrue(optimizedPlan, optimizedPlan.contains("PartitionPruned: true, PartitionNums: 2"));
@@ -283,7 +283,7 @@ public class HiveTableSourceTest {
 			assertEquals("[1, 2]", results.toString());
 
 			query = tableEnv.sqlQuery("select x from db1.part where p2 = 'c:2' order by x");
-			explain = tableEnv.explain(query).split("==.*==\n");
+			explain = query.explain().split("==.*==\n");
 			assertFalse(catalog.fallback);
 			optimizedPlan = explain[2];
 			assertTrue(optimizedPlan, optimizedPlan.contains("PartitionPruned: true, PartitionNums: 1"));
@@ -314,7 +314,7 @@ public class HiveTableSourceTest {
 
 			Table query = tableEnv.sqlQuery(
 					"select x from db1.part where p1>cast('2018-08-09' as date) and p2<>cast('2018-08-08 08:08:09' as timestamp)");
-			String[] explain = tableEnv.explain(query).split("==.*==\n");
+			String[] explain = query.explain().split("==.*==\n");
 			assertTrue(catalog.fallback);
 			String optimizedPlan = explain[2];
 			assertTrue(optimizedPlan, optimizedPlan.contains("PartitionPruned: true, PartitionNums: 1"));
@@ -341,7 +341,7 @@ public class HiveTableSourceTest {
 			TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode();
 			tableEnv.registerCatalog(catalogName, hiveCatalog);
 			Table table = tableEnv.sqlQuery("select p1, count(y) from hive.`default`.src group by p1");
-			String[] explain = tableEnv.explain(table).split("==.*==\n");
+			String[] explain = table.explain().split("==.*==\n");
 			assertEquals(4, explain.length);
 			String logicalPlan = explain[2];
 			String physicalPlan = explain[3];
@@ -375,7 +375,7 @@ public class HiveTableSourceTest {
 			TableEnvironment tableEnv = HiveTestUtils.createTableEnvWithBlinkPlannerBatchMode();
 			tableEnv.registerCatalog(catalogName, hiveCatalog);
 			Table table = tableEnv.sqlQuery("select * from hive.`default`.src limit 1");
-			String[] explain = tableEnv.explain(table).split("==.*==\n");
+			String[] explain = table.explain().split("==.*==\n");
 			assertEquals(4, explain.length);
 			String logicalPlan = explain[2];
 			String physicalPlan = explain[3];
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
index fc3f6fb..9aca18c 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/connectors/hive/TableEnvHiveConnectorTest.java
@@ -19,7 +19,9 @@
 package org.apache.flink.connectors.hive;
 
 import org.apache.flink.table.HiveVersionTestUtil;
+import org.apache.flink.table.api.StatementSet;
 import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.api.constraints.UniqueConstraint;
 import org.apache.flink.table.api.internal.TableImpl;
@@ -30,6 +32,7 @@ import org.apache.flink.table.catalog.hive.HiveTestUtils;
 import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientFactory;
 import org.apache.flink.table.catalog.hive.client.HiveMetastoreClientWrapper;
 import org.apache.flink.table.catalog.hive.client.HiveShimLoader;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.ArrayUtils;
 
@@ -94,8 +97,7 @@ public class TableEnvHiveConnectorTest {
 		TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
 
 		// test generating partitions with default name
-		tableEnv.sqlUpdate("insert into db1.part select * from db1.src");
-		tableEnv.execute("mytest");
+		TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.part select * from db1.src");
 		HiveConf hiveConf = hiveShell.getHiveConf();
 		String defaultPartName = hiveConf.getVar(HiveConf.ConfVars.DEFAULTPARTITIONNAME);
 		Table hiveTable = hmsClient.getTable("db1", "part");
@@ -119,8 +121,7 @@ public class TableEnvHiveConnectorTest {
 		TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
 
 		// just make sure the query runs through, no need to verify result
-		tableEnv.sqlUpdate("insert into db1.dest select count(d) from db1.src");
-		tableEnv.execute("test");
+		TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest select count(d) from db1.src");
 
 		hiveShell.execute("drop database db1 cascade");
 	}
@@ -187,8 +188,7 @@ public class TableEnvHiveConnectorTest {
 		verifyFlinkQueryResult(tableEnv.sqlQuery("select * from db1.src"), expected);
 
 		// populate dest table with source table
-		tableEnv.sqlUpdate("insert into db1.dest select * from db1.src");
-		tableEnv.execute("test_" + format);
+		TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest select * from db1.src");
 
 		// verify data on hive side
 		verifyHiveQueryResult("select * from db1.dest", expected);
@@ -219,15 +219,13 @@ public class TableEnvHiveConnectorTest {
 
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
 			// populate src2 with same data from Flink
-			tableEnv.sqlUpdate("insert into db1.src2 values (cast(1.0 as decimal(10,2))), (cast(2.12 as decimal(10,2))), " +
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.src2 values (cast(1.0 as decimal(10,2))), (cast(2.12 as decimal(10,2))), " +
 					"(cast(5.123 as decimal(10,2))), (cast(5.456 as decimal(10,2))), (cast(123456789.12 as decimal(10,2)))");
-			tableEnv.execute("test1");
 			// verify src1 and src2 contain same data
 			verifyHiveQueryResult("select * from db1.src2", hiveShell.executeQuery("select * from db1.src1"));
 
 			// populate dest with src1 from Flink -- to test reading decimal type from Hive
-			tableEnv.sqlUpdate("insert into db1.dest select * from db1.src1");
-			tableEnv.execute("test2");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest select * from db1.src1");
 			verifyHiveQueryResult("select * from db1.dest", hiveShell.executeQuery("select * from db1.src1"));
 		} finally {
 			hiveShell.execute("drop database db1 cascade");
@@ -243,8 +241,7 @@ public class TableEnvHiveConnectorTest {
 			HiveTestUtils.createTextTableInserter(hiveShell, "db1", "dest").addRow(new Object[]{1, "a"}).addRow(new Object[]{2, "b"}).commit();
 			verifyHiveQueryResult("select * from db1.dest", Arrays.asList("1\ta", "2\tb"));
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert overwrite db1.dest values (3, 'c')");
-			tableEnv.execute("test insert overwrite");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert overwrite db1.dest values (3, 'c')");
 			verifyHiveQueryResult("select * from db1.dest", Collections.singletonList("3\tc"));
 
 			// static partition
@@ -252,14 +249,12 @@ public class TableEnvHiveConnectorTest {
 			HiveTestUtils.createTextTableInserter(hiveShell, "db1", "part").addRow(new Object[]{1}).commit("y=1");
 			HiveTestUtils.createTextTableInserter(hiveShell, "db1", "part").addRow(new Object[]{2}).commit("y=2");
 			tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert overwrite db1.part partition (y=1) select 100");
-			tableEnv.execute("insert overwrite static partition");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert overwrite db1.part partition (y=1) select 100");
 			verifyHiveQueryResult("select * from db1.part", Arrays.asList("100\t1", "2\t2"));
 
 			// dynamic partition
 			tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert overwrite db1.part values (200,2),(3,3)");
-			tableEnv.execute("insert overwrite dynamic partition");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert overwrite db1.part values (200,2),(3,3)");
 			// only overwrite dynamically matched partitions, other existing partitions remain intact
 			verifyHiveQueryResult("select * from db1.part", Arrays.asList("100\t1", "200\t2", "3\t3"));
 		} finally {
@@ -275,8 +270,7 @@ public class TableEnvHiveConnectorTest {
 			HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src").addRow(new Object[]{1}).addRow(new Object[]{2}).commit();
 			hiveShell.execute("create table db1.dest (x int) partitioned by (p1 string, p2 double)");
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert into db1.dest partition (p1='1''1', p2=1.1) select x from db1.src");
-			tableEnv.execute("static partitioning");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest partition (p1='1''1', p2=1.1) select x from db1.src");
 			assertEquals(1, hiveCatalog.listPartitions(new ObjectPath("db1", "dest")).size());
 			verifyHiveQueryResult("select * from db1.dest", Arrays.asList("1\t1'1\t1.1", "2\t1'1\t1.1"));
 		} finally {
@@ -296,8 +290,7 @@ public class TableEnvHiveConnectorTest {
 					.commit();
 			hiveShell.execute("create table db1.dest (x int) partitioned by (p1 string, p2 double)");
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert into db1.dest select * from db1.src");
-			tableEnv.execute("dynamic partitioning");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest select * from db1.src");
 			assertEquals(3, hiveCatalog.listPartitions(new ObjectPath("db1", "dest")).size());
 			verifyHiveQueryResult("select * from db1.dest", Arrays.asList("1\ta\t1.1", "2\ta\t2.2", "3\tb\t3.3"));
 		} finally {
@@ -313,8 +306,7 @@ public class TableEnvHiveConnectorTest {
 			HiveTestUtils.createTextTableInserter(hiveShell, "db1", "src").addRow(new Object[]{1, "a"}).addRow(new Object[]{2, "b"}).commit();
 			hiveShell.execute("create table db1.dest (x int) partitioned by (p1 double, p2 string)");
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert into db1.dest partition (p1=1.1) select x,y from db1.src");
-			tableEnv.execute("partial dynamic partitioning");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest partition (p1=1.1) select x,y from db1.src");
 			assertEquals(2, hiveCatalog.listPartitions(new ObjectPath("db1", "dest")).size());
 			verifyHiveQueryResult("select * from db1.dest", Arrays.asList("1\t1.1\ta", "2\t1.1\tb"));
 		} finally {
@@ -341,8 +333,7 @@ public class TableEnvHiveConnectorTest {
 			results = Lists.newArrayList(tableEnv.sqlQuery("select x from db1.part where dt=cast('2019-12-25' as date)").execute().collect());
 			assertEquals("[3]", results.toString());
 
-			tableEnv.sqlUpdate("insert into db1.part select 4,cast('2019-12-31' as date),cast('2019-12-31 12:00:00.0' as timestamp)");
-			tableEnv.execute("insert");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.part select 4,cast('2019-12-31' as date),cast('2019-12-31 12:00:00.0' as timestamp)");
 			results = Lists.newArrayList(tableEnv.sqlQuery("select max(dt) from db1.part").execute().collect());
 			assertEquals("[2019-12-31]", results.toString());
 		} finally {
@@ -461,8 +452,7 @@ public class TableEnvHiveConnectorTest {
 			assertEquals(LocalDateTime.of(2019, 11, 11, 0, 0), results.get(0).getField(0));
 			assertEquals(LocalDateTime.of(2019, 12, 3, 15, 43, 32, 123456789), results.get(1).getField(0));
 			// test write timestamp to hive
-			tableEnv.sqlUpdate("insert into db1.dest select max(ts) from db1.src");
-			tableEnv.execute("write timestamp to hive");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest select max(ts) from db1.src");
 			verifyHiveQueryResult("select * from db1.dest", Collections.singletonList("2019-12-03 15:43:32.123456789"));
 		} finally {
 			hiveShell.execute("drop database db1 cascade");
@@ -486,8 +476,7 @@ public class TableEnvHiveConnectorTest {
 			assertEquals(LocalDate.of(2019, 12, 9), results.get(0).getField(0));
 			assertEquals(LocalDate.of(2019, 12, 12), results.get(1).getField(0));
 			// test write date to hive
-			tableEnv.sqlUpdate("insert into db1.dest select max(dt) from db1.src");
-			tableEnv.execute("write date to hive");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into db1.dest select max(dt) from db1.src");
 			verifyHiveQueryResult("select * from db1.dest", Collections.singletonList("2019-12-12"));
 		} finally {
 			hiveShell.execute("drop database db1 cascade");
@@ -535,9 +524,12 @@ public class TableEnvHiveConnectorTest {
 		try {
 			hiveShell.execute("create table db1.dest (x int) partitioned by (p string)");
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert into db1.dest select 1,'  '");
-			tableEnv.sqlUpdate("insert into db1.dest select 2,'a \t'");
-			tableEnv.execute("insert");
+			StatementSet stmtSet = tableEnv.createStatementSet();
+			stmtSet.addInsertSql("insert into db1.dest select 1,'  '");
+			stmtSet.addInsertSql("insert into db1.dest select 2,'a \t'");
+			TableResult tableResult = stmtSet.execute();
+			// wait job finished
+			tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 			assertEquals("[p=  , p=a %09]", hiveShell.executeQuery("show partitions db1.dest").toString());
 		} finally {
 			hiveShell.execute("drop database db1 cascade");
@@ -556,8 +548,7 @@ public class TableEnvHiveConnectorTest {
 					.commit();
 			hiveCatalog.getHiveConf().setBoolVar(HiveConf.ConfVars.COMPRESSRESULT, true);
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert overwrite db1.dest select * from db1.src");
-			tableEnv.execute("insert dest");
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert overwrite db1.dest select * from db1.src");
 			List<String> expected = Arrays.asList("a\tb", "c\td");
 			verifyHiveQueryResult("select * from db1.dest", expected);
 			verifyFlinkQueryResult(tableEnv.sqlQuery("select * from db1.dest"), expected);
@@ -590,11 +581,9 @@ public class TableEnvHiveConnectorTest {
 		try {
 			hiveShell.execute("create table db1.dest (x int) partitioned by (p string) stored as rcfile");
 			TableEnvironment tableEnv = getTableEnvWithHiveCatalog();
-			tableEnv.sqlUpdate("insert overwrite db1.dest partition (p='1') select 1");
-			tableEnv.execute(null);
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert overwrite db1.dest partition (p='1') select 1");
 			hiveShell.execute("alter table db1.dest set fileformat sequencefile");
-			tableEnv.sqlUpdate("insert overwrite db1.dest partition (p='1') select 1");
-			tableEnv.execute(null);
+			TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert overwrite db1.dest partition (p='1') select 1");
 			assertEquals("[1,1]", Lists.newArrayList(tableEnv.sqlQuery("select * from db1.dest").execute().collect()).toString());
 		} finally {
 			hiveShell.execute("drop database db1 cascade");
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java
index 870db25..e1c27df 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogITCase.java
@@ -34,6 +34,7 @@ import org.apache.flink.table.catalog.ObjectPath;
 import org.apache.flink.table.descriptors.FileSystem;
 import org.apache.flink.table.descriptors.FormatDescriptor;
 import org.apache.flink.table.descriptors.OldCsv;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
 import org.apache.flink.types.Row;
 import org.apache.flink.util.FileUtils;
 
@@ -106,7 +107,7 @@ public class HiveCatalogITCase {
 
 		String path = this.getClass().getResource("/csv/test.csv").getPath();
 
-		tableEnv.sqlUpdate("create table test2 (name String, age Int) with (\n" +
+		tableEnv.executeSql("create table test2 (name String, age Int) with (\n" +
 			"   'connector.type' = 'filesystem',\n" +
 			"   'connector.path' = 'file://" + path + "',\n" +
 			"   'format.type' = 'csv'\n" +
@@ -125,7 +126,7 @@ public class HiveCatalogITCase {
 			new HashSet<>(result)
 		);
 
-		tableEnv.sqlUpdate("ALTER TABLE test2 RENAME TO newtable");
+		tableEnv.executeSql("ALTER TABLE test2 RENAME TO newtable");
 
 		t = tableEnv.sqlQuery("SELECT * FROM myhive.`default`.newtable");
 
@@ -140,7 +141,7 @@ public class HiveCatalogITCase {
 			new HashSet<>(result)
 		);
 
-		tableEnv.sqlUpdate("DROP TABLE newtable");
+		tableEnv.executeSql("DROP TABLE newtable");
 	}
 
 	@Test
@@ -208,11 +209,10 @@ public class HiveCatalogITCase {
 			result
 		);
 
-		tableEnv.sqlUpdate(
+		TableEnvUtil.execInsertSqlAndWaitResult(tableEnv,
 			String.format("insert into myhive.`default`.%s select * from myhive.`default`.%s",
 				sinkTableName,
 				sourceTableName));
-		tableEnv.execute("myjob");
 
 		// assert written result
 		File resultFile = new File(p.toAbsolutePath().toString());
@@ -226,8 +226,8 @@ public class HiveCatalogITCase {
 		// No more line
 		assertNull(reader.readLine());
 
-		tableEnv.sqlUpdate(String.format("DROP TABLE %s", sourceTableName));
-		tableEnv.sqlUpdate(String.format("DROP TABLE %s", sinkTableName));
+		tableEnv.executeSql(String.format("DROP TABLE %s", sourceTableName));
+		tableEnv.executeSql(String.format("DROP TABLE %s", sinkTableName));
 	}
 
 	@Test
@@ -242,22 +242,20 @@ public class HiveCatalogITCase {
 
 		String srcPath = this.getClass().getResource("/csv/test3.csv").getPath();
 
-		tableEnv.sqlUpdate("CREATE TABLE src (" +
+		tableEnv.executeSql("CREATE TABLE src (" +
 				"price DECIMAL(10, 2),currency STRING,ts6 TIMESTAMP(6),ts AS CAST(ts6 AS TIMESTAMP(3)),WATERMARK FOR ts AS ts) " +
 				String.format("WITH ('connector.type' = 'filesystem','connector.path' = 'file://%s','format.type' = 'csv')", srcPath));
 
 		String sinkPath = new File(tempFolder.newFolder(), "csv-order-sink").toURI().toString();
 
-		tableEnv.sqlUpdate("CREATE TABLE sink (" +
+		tableEnv.executeSql("CREATE TABLE sink (" +
 				"window_end TIMESTAMP(3),max_ts TIMESTAMP(6),counter BIGINT,total_price DECIMAL(10, 2)) " +
 				String.format("WITH ('connector.type' = 'filesystem','connector.path' = '%s','format.type' = 'csv')", sinkPath));
 
-		tableEnv.sqlUpdate("INSERT INTO sink " +
+		TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "INSERT INTO sink " +
 				"SELECT TUMBLE_END(ts, INTERVAL '5' SECOND),MAX(ts6),COUNT(*),MAX(price) FROM src " +
 				"GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)");
 
-		tableEnv.execute("testJob");
-
 		String expected = "2019-12-12 00:00:05.0,2019-12-12 00:00:04.004001,3,50.00\n" +
 				"2019-12-12 00:00:10.0,2019-12-12 00:00:06.006001,2,5.33\n";
 		assertEquals(expected, FileUtils.readFileUtf8(new File(new URI(sinkPath))));
diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java
index dd280f6..d0a0fc6 100644
--- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java
+++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/HiveCatalogUseBlinkITCase.java
@@ -40,6 +40,7 @@ import org.apache.flink.table.functions.hive.util.TestHiveGenericUDF;
 import org.apache.flink.table.functions.hive.util.TestHiveSimpleUDF;
 import org.apache.flink.table.functions.hive.util.TestHiveUDTF;
 import org.apache.flink.table.planner.runtime.utils.BatchTestBase;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
 import org.apache.flink.table.planner.runtime.utils.TestingRetractSink;
 import org.apache.flink.table.util.JavaScalaConversionUtil;
 import org.apache.flink.test.util.AbstractTestBase;
@@ -212,8 +213,8 @@ public class HiveCatalogUseBlinkITCase extends AbstractTestBase {
 					false
 			);
 
-			tEnv.sqlUpdate(format("insert into %s " + selectSql, sinkTableName));
-			tEnv.execute("myjob");
+			TableEnvUtil.execInsertSqlAndWaitResult(
+					tEnv, format("insert into %s " + selectSql, sinkTableName));
 
 			// assert written result
 			StringBuilder builder = new StringBuilder();
diff --git a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java
index a3d0acd..cdbca87 100644
--- a/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java
+++ b/flink-connectors/flink-connector-kafka-base/src/test/java/org/apache/flink/streaming/connectors/kafka/KafkaTableTestBase.java
@@ -28,6 +28,7 @@ import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.table.catalog.CatalogTableImpl;
 import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
 import org.apache.flink.types.Row;
 
 import org.junit.Test;
@@ -127,9 +128,7 @@ public abstract class KafkaTableTestBase extends KafkaTestBase {
 			"  (5.33,'US Dollar','2019-12-12 00:00:05.006001'), \n" +
 			"  (0,'DUMMY','2019-12-12 00:00:10'))\n" +
 			"  AS orders (price, currency, ts)";
-		tEnv.sqlUpdate(initialValues);
-
-		tEnv.execute("Job_1");
+		TableEnvUtil.execInsertSqlAndWaitResult(tEnv, initialValues);
 
 		// ---------- Consume stream from Kafka -------------------
 
diff --git a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java
index 86621fa..a1b1fef 100644
--- a/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java
+++ b/flink-connectors/flink-hbase/src/test/java/org/apache/flink/addons/hbase/HBaseConnectorITCase.java
@@ -39,6 +39,7 @@ import org.apache.flink.table.descriptors.DescriptorProperties;
 import org.apache.flink.table.factories.TableFactoryService;
 import org.apache.flink.table.functions.ScalarFunction;
 import org.apache.flink.table.planner.runtime.utils.BatchTableEnvUtil;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
 import org.apache.flink.table.planner.sinks.CollectRowTableSink;
 import org.apache.flink.table.planner.sinks.CollectTableSink;
 import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
@@ -292,10 +293,7 @@ public class HBaseConnectorITCase extends HBaseTestBase {
 		tEnv.registerTableSink("hbase", tableSink);
 
 		String query = "INSERT INTO hbase SELECT ROW(f1c1), ROW(f2c1, f2c2), rowkey, ROW(f3c1, f3c2, f3c3) FROM src";
-		tEnv.sqlUpdate(query);
-
-		// wait to finish
-		tEnv.execute("HBase Job");
+		TableEnvUtil.execInsertSqlAndWaitResult(tEnv, query);
 
 		// start a batch scan job to verify contents in HBase table
 		// start a batch scan job to verify contents in HBase table
@@ -360,19 +358,16 @@ public class HBaseConnectorITCase extends HBaseTestBase {
 			"    'connector.zookeeper.quorum' = '" + quorum + "',\n" +
 			"    'connector.zookeeper.znode.parent' = '/hbase' " +
 			")";
-		tEnv.sqlUpdate(ddl);
+		tEnv.executeSql(ddl);
 
 		String query = "INSERT INTO hbase " +
 			"SELECT rowkey, ROW(f1c1), ROW(f2c1, f2c2), ROW(f3c1, f3c2, f3c3), ROW(f4c1, f4c2, f4c3) " +
 			"FROM src";
-		tEnv.sqlUpdate(query);
-
-		// wait to finish
-		tEnv.execute("HBase Job");
+		TableEnvUtil.execInsertSqlAndWaitResult(tEnv, query);
 
 		// start a batch scan job to verify contents in HBase table
 		TableEnvironment batchTableEnv = createBatchTableEnv();
-		batchTableEnv.sqlUpdate(ddl);
+		batchTableEnv.executeSql(ddl);
 
 		Table table = batchTableEnv.sqlQuery(
 			"SELECT " +
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTypeTest.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTypeTest.java
index 9f3ff53..8b2a66c 100644
--- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTypeTest.java
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCDataTypeTest.java
@@ -149,7 +149,7 @@ public class JDBCDataTypeTest {
 				.build();
 		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
 
-		tEnv.sqlUpdate(sqlDDL);
+		tEnv.executeSql(sqlDDL);
 
 		if (testItem.expectError != null) {
 			try {
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
index 5c8c4a3..b291e2f 100644
--- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCTableSourceITCase.java
@@ -90,7 +90,7 @@ public class JDBCTableSourceITCase extends AbstractTestBase {
 			.build();
 		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
 
-		tEnv.sqlUpdate(
+		tEnv.executeSql(
 			"CREATE TABLE " + INPUT_TABLE + "(" +
 				"id BIGINT," +
 				"timestamp6_col TIMESTAMP(6)," +
@@ -127,7 +127,7 @@ public class JDBCTableSourceITCase extends AbstractTestBase {
 				.build();
 		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env, envSettings);
 
-		tEnv.sqlUpdate(
+		tEnv.executeSql(
 			"CREATE TABLE " + INPUT_TABLE + "(" +
 				"id BIGINT," +
 				"timestamp6_col TIMESTAMP(6)," +
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java
index f5ca948..f86d56a 100644
--- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/JDBCUpsertTableSinkITCase.java
@@ -26,6 +26,7 @@ import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExt
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.api.java.StreamTableEnvironment;
 import org.apache.flink.test.util.AbstractTestBase;
 import org.apache.flink.types.Row;
@@ -132,7 +133,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
 		StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
 
-		tEnv.sqlUpdate(
+		tEnv.executeSql(
 				"CREATE TABLE upsertSink (" +
 						"  real_data float" +
 						") WITH (" +
@@ -141,8 +142,9 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 						"  'connector.table'='REAL_TABLE'" +
 						")");
 
-		tEnv.sqlUpdate("INSERT INTO upsertSink SELECT CAST(1.0 as FLOAT)");
-		tEnv.execute("job name");
+		TableResult tableResult = tEnv.executeSql("INSERT INTO upsertSink SELECT CAST(1.0 as FLOAT)");
+		// wait to finish
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 		check(new Row[] {Row.of(1.0f)}, DB_URL, "REAL_TABLE", new String[]{"real_data"});
 	}
 
@@ -162,7 +164,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 			}), $("id"), $("num"), $("text"), $("ts"));
 
 		tEnv.createTemporaryView("T", t);
-		tEnv.sqlUpdate(
+		tEnv.executeSql(
 			"CREATE TABLE upsertSink (" +
 				"  cnt BIGINT," +
 				"  lencnt BIGINT," +
@@ -174,7 +176,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 				"  'connector.table'='" + OUTPUT_TABLE1 + "'" +
 				")");
 
-		tEnv.sqlUpdate("INSERT INTO upsertSink \n" +
+		TableResult tableResult = tEnv.executeSql("INSERT INTO upsertSink \n" +
 			"SELECT cnt, COUNT(len) AS lencnt, cTag, MAX(ts) AS ts\n" +
 			"FROM (\n" +
 			"  SELECT len, COUNT(id) as cnt, cTag, MAX(ts) AS ts\n" +
@@ -182,7 +184,8 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 			"  GROUP BY len, cTag\n" +
 			")\n" +
 			"GROUP BY cnt, cTag");
-		tEnv.execute("job name");
+		// wait to finish
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 		check(new Row[] {
 				Row.of(1, 5, 1, Timestamp.valueOf("1970-01-01 00:00:00.006")),
 				Row.of(7, 1, 1, Timestamp.valueOf("1970-01-01 00:00:00.021")),
@@ -201,7 +204,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 
 		tEnv.registerTable("T", t);
 
-		tEnv.sqlUpdate(
+		tEnv.executeSql(
 			"CREATE TABLE upsertSink (" +
 				"  id INT," +
 				"  num BIGINT," +
@@ -212,8 +215,10 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 				"  'connector.table'='" + OUTPUT_TABLE2 + "'" +
 				")");
 
-		tEnv.sqlUpdate("INSERT INTO upsertSink SELECT id, num, ts FROM T WHERE id IN (2, 10, 20)");
-		tEnv.execute("job name");
+		TableResult tableResult = tEnv.executeSql(
+				"INSERT INTO upsertSink SELECT id, num, ts FROM T WHERE id IN (2, 10, 20)");
+		// wait to finish
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 		check(new Row[] {
 				Row.of(2, 2, Timestamp.valueOf("1970-01-01 00:00:00.002")),
 				Row.of(10, 4, Timestamp.valueOf("1970-01-01 00:00:00.01")),
@@ -227,7 +232,7 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 				.useBlinkPlanner().inBatchMode().build();
 		TableEnvironment tEnv = TableEnvironment.create(bsSettings);
 
-		tEnv.sqlUpdate(
+		tEnv.executeSql(
 			"CREATE TABLE USER_RESULT(" +
 				"NAME VARCHAR," +
 				"SCORE BIGINT" +
@@ -237,12 +242,13 @@ public class JDBCUpsertTableSinkITCase extends AbstractTestBase {
 				"'connector.table' = '" + OUTPUT_TABLE3 + "'" +
 				")");
 
-		tEnv.sqlUpdate("INSERT INTO USER_RESULT\n" +
+		TableResult tableResult  = tEnv.executeSql("INSERT INTO USER_RESULT\n" +
 				"SELECT user_name, score " +
 				"FROM (VALUES (1, 'Bob'), (22, 'Tom'), (42, 'Kim'), " +
 				"(42, 'Kim'), (1, 'Bob')) " +
 				"AS UserCountTable(score, user_name)");
-		tEnv.execute("test");
+		// wait to finish
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 
 		check(new Row[] {
 				Row.of("Bob", 1),
diff --git a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java
index b86f7ab..894bed1 100644
--- a/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java
+++ b/flink-connectors/flink-jdbc/src/test/java/org/apache/flink/api/java/io/jdbc/catalog/PostgresCatalogITCase.java
@@ -20,6 +20,7 @@ package org.apache.flink.api.java.io.jdbc.catalog;
 
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.types.Row;
 
 import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
@@ -71,8 +72,9 @@ public class PostgresCatalogITCase extends PostgresCatalogTestBase {
 	public void test_insert() throws Exception {
 		TableEnvironment tEnv = getTableEnvWithPgCatalog();
 
-		tEnv.sqlUpdate(String.format("insert into %s select * from `%s`", TABLE4, TABLE1));
-		tEnv.execute("test");
+		TableResult tableResult = tEnv.executeSql(String.format("insert into %s select * from `%s`", TABLE4, TABLE1));
+		// wait to finish
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 
 		List<Row> results = Lists.newArrayList(
 			tEnv.sqlQuery(String.format("select * from %s", TABLE1)).execute().collect());
diff --git a/flink-end-to-end-tests/flink-batch-sql-test/src/main/java/org/apache/flink/sql/tests/BatchSQLTestProgram.java b/flink-end-to-end-tests/flink-batch-sql-test/src/main/java/org/apache/flink/sql/tests/BatchSQLTestProgram.java
index 615b6d6..903bc8c 100644
--- a/flink-end-to-end-tests/flink-batch-sql-test/src/main/java/org/apache/flink/sql/tests/BatchSQLTestProgram.java
+++ b/flink-end-to-end-tests/flink-batch-sql-test/src/main/java/org/apache/flink/sql/tests/BatchSQLTestProgram.java
@@ -26,6 +26,7 @@ import org.apache.flink.api.java.utils.ParameterTool;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.api.TableSchema;
 import org.apache.flink.table.sinks.CsvTableSink;
 import org.apache.flink.table.sources.InputFormatTableSource;
@@ -66,8 +67,9 @@ public class BatchSQLTestProgram {
 			new CsvTableSink(outputPath)
 				.configure(new String[]{"f0", "f1"}, new TypeInformation[]{Types.INT, Types.SQL_TIMESTAMP}));
 
-		tEnv.sqlUpdate(sqlStatement);
-		tEnv.execute("TestSqlJob");
+		TableResult result = tEnv.executeSql(sqlStatement);
+		// wait job finish
+		result.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 	}
 
 	/**
diff --git a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
index 966c364..ac1d6c9 100644
--- a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
+++ b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
@@ -23,6 +23,7 @@ import org.apache.flink.core.fs.FileSystem;
 import org.apache.flink.table.api.EnvironmentSettings;
 import org.apache.flink.table.api.Table;
 import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.api.config.ExecutionConfigOptions;
 import org.apache.flink.table.api.config.OptimizerConfigOptions;
 import org.apache.flink.table.catalog.ConnectorCatalogTable;
@@ -100,8 +101,11 @@ public class TpcdsTestProgram {
 						resultTable.getSchema().getFieldNames(),
 						resultTable.getSchema().getFieldDataTypes()
 					));
-			tableEnvironment.insertInto(resultTable, sinkTableName);
-			tableEnvironment.execute(queryName);
+			TableResult tableResult = resultTable.executeInsert(sinkTableName);
+			// wait job finish
+			tableResult.getJobClient().get()
+					.getJobExecutionResult(Thread.currentThread().getContextClassLoader())
+					.get();
 			System.out.println("[INFO]Run TPC-DS query " + queryId + " success.");
 		}
 	}
diff --git a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamWindowSQLExample.java b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamWindowSQLExample.java
index 4620a8a5..b758150 100644
--- a/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamWindowSQLExample.java
+++ b/flink-examples/flink-examples-table/src/main/java/org/apache/flink/table/examples/java/StreamWindowSQLExample.java
@@ -66,7 +66,7 @@ public class StreamWindowSQLExample {
 			"  'connector.path' = '" + path + "',\n" +
 			"  'format.type' = 'csv'\n" +
 			")";
-		tEnv.sqlUpdate(ddl);
+		tEnv.executeSql(ddl);
 
 		// run a SQL query on the table and retrieve the result as a new Table
 		String query = "SELECT\n" +
diff --git a/flink-python/pyflink/table/table.py b/flink-python/pyflink/table/table.py
index a6f8ff2..fefc1f7 100644
--- a/flink-python/pyflink/table/table.py
+++ b/flink-python/pyflink/table/table.py
@@ -16,6 +16,8 @@
 # limitations under the License.
 ################################################################################
 
+import warnings
+
 from py4j.java_gateway import get_method
 from pyflink.java_gateway import get_gateway
 from pyflink.table.table_schema import TableSchema
@@ -680,7 +682,14 @@ class Table(object):
         :param table_path: The path of the registered :class:`~pyflink.table.TableSink` to which
                the :class:`~pyflink.table.Table` is written.
         :type table_path: str
+
+        .. note:: Deprecated in 1.11. Use :func:`execute_insert` for single sink,
+                  use :class:`TableTableEnvironment`#:func:`create_statement_set`
+                  for multiple sinks.
         """
+        warnings.warn("Deprecated in 1.11. Use execute_insert for single sink, "
+                      "use TableTableEnvironment#create_statement_set for multiple sinks.",
+                      DeprecationWarning)
         self._j_table.insertInto(table_path)
 
     def get_schema(self):
diff --git a/flink-python/pyflink/table/table_environment.py b/flink-python/pyflink/table/table_environment.py
index 656c960..af2e7f5 100644
--- a/flink-python/pyflink/table/table_environment.py
+++ b/flink-python/pyflink/table/table_environment.py
@@ -312,7 +312,12 @@ class TableEnvironment(object):
         .. versionchanged:: 1.10.0
             The signature is changed, e.g. the parameter *table_path_continued* was removed and
             the parameter *target_path* is moved before the parameter *table*.
+
+        .. note:: Deprecated in 1.11. Use :func:`execute_insert` for single sink,
+                  use :func:`create_statement_set` for multiple sinks.
         """
+        warnings.warn("Deprecated in 1.11. Use execute_insert for single sink,"
+                      "use create_statement_set for multiple sinks.", DeprecationWarning)
         self._j_tenv.insertInto(target_path, table._j_table)
 
     def list_catalogs(self):
@@ -464,7 +469,10 @@ class TableEnvironment(object):
         :type extended: bool
         :return: The table for which the AST and execution plan will be returned.
         :rtype: str
+
+        .. note:: Deprecated in 1.11. Use :class:`Table`#:func:`explain` instead.
         """
+        warnings.warn("Deprecated in 1.11. Use Table#explain instead.", DeprecationWarning)
         if table is None:
             return self._j_tenv.explain(extended)
         else:
@@ -608,7 +616,12 @@ class TableEnvironment(object):
 
         :param stmt: The SQL statement to evaluate.
         :type stmt: str
+
+        .. note:: Deprecated in 1.11. Use :func:`execute_sql` for single statement,
+                  use :func:`create_statement_set` for multiple DML statements.
         """
+        warnings.warn("Deprecated in 1.11. Use execute_sql for single statement, "
+                      "use create_statement_set for multiple DML statements.", DeprecationWarning)
         self._j_tenv.sqlUpdate(stmt)
 
     def get_current_catalog(self):
@@ -1022,7 +1035,12 @@ class TableEnvironment(object):
         :param job_name: Desired name of the job.
         :type job_name: str
         :return: The result of the job execution, containing elapsed time and accumulators.
+
+        .. note:: Deprecated in 1.11. Use :func:`execute_sql` for single sink,
+                  use :func:`create_statement_set` for multiple sinks.
         """
+        warnings.warn("Deprecated in 1.11. Use execute_sql for single sink, "
+                      "use create_statement_set for multiple sinks.", DeprecationWarning)
         jvm = get_gateway().jvm
         jars_key = jvm.org.apache.flink.configuration.PipelineOptions.JARS.key()
         classpaths_key = jvm.org.apache.flink.configuration.PipelineOptions.CLASSPATHS.key()
diff --git a/flink-python/pyflink/table/tests/test_sql.py b/flink-python/pyflink/table/tests/test_sql.py
index c561760..61de9b1 100644
--- a/flink-python/pyflink/table/tests/test_sql.py
+++ b/flink-python/pyflink/table/tests/test_sql.py
@@ -35,7 +35,7 @@ class SqlTests(object):
         self.t_env.sql_update("create temporary function func1 as "
                               "'pyflink.table.tests.test_udf.add' language python")
         table = self.t_env.from_elements([(1, 2)]).alias("a, b").select("func1(a, b)")
-        plan = self.t_env.explain(table)
+        plan = table.explain()
         self.assertTrue(plan.find("PythonCalc(select=[add(f0, f1) AS _c0])") >= 0)
 
 
diff --git a/flink-python/pyflink/table/tests/test_table_environment_api.py b/flink-python/pyflink/table/tests/test_table_environment_api.py
index 5219efb..87215e3 100644
--- a/flink-python/pyflink/table/tests/test_table_environment_api.py
+++ b/flink-python/pyflink/table/tests/test_table_environment_api.py
@@ -56,7 +56,7 @@ class TableEnvironmentTest(object):
         t = t_env.from_elements([], schema)
         result = t.select("1 + a, b, c")
 
-        actual = t_env.explain(result)
+        actual = result.explain()
 
         assert isinstance(actual, str)
 
@@ -69,7 +69,7 @@ class TableEnvironmentTest(object):
         t = t_env.from_elements([], schema)
         result = t.select("1 + a, b, c")
 
-        actual = t_env.explain(result, True)
+        actual = result.explain(ExplainDetail.ESTIMATED_COST, ExplainDetail.CHANGELOG_MODE)
 
         assert isinstance(actual, str)
 
@@ -256,10 +256,11 @@ class StreamTableEnvironmentTests(TableEnvironmentTest, PyFlinkStreamTableTestCa
             "sink2",
             source_sink_utils.TestAppendSink(field_names, field_types))
 
-        t_env.sql_update("insert into sink1 select * from %s where a > 100" % source)
-        t_env.sql_update("insert into sink2 select * from %s where a < 100" % source)
+        stmt_set = t_env.create_statement_set()
+        stmt_set.add_insert_sql("insert into sink1 select * from %s where a > 100" % source)
+        stmt_set.add_insert_sql("insert into sink2 select * from %s where a < 100" % source)
 
-        actual = t_env.explain(extended=True)
+        actual = stmt_set.explain(ExplainDetail.ESTIMATED_COST, ExplainDetail.CHANGELOG_MODE)
         assert isinstance(actual, str)
 
     def test_explain_sql_without_explain_detail(self):
@@ -444,10 +445,11 @@ class BatchTableEnvironmentTests(TableEnvironmentTest, PyFlinkBatchTableTestCase
             "sink2",
             CsvTableSink(field_names, field_types, "path2"))
 
-        t_env.sql_update("insert into sink1 select * from %s where a > 100" % source)
-        t_env.sql_update("insert into sink2 select * from %s where a < 100" % source)
+        stmt_set = t_env.create_statement_set()
+        stmt_set.add_insert_sql("insert into sink1 select * from %s where a > 100" % source)
+        stmt_set.add_insert_sql("insert into sink2 select * from %s where a < 100" % source)
 
-        actual = t_env.explain(extended=True)
+        actual = stmt_set.explain(ExplainDetail.ESTIMATED_COST, ExplainDetail.CHANGELOG_MODE)
 
         assert isinstance(actual, str)
 
@@ -548,10 +550,11 @@ class BlinkBatchTableEnvironmentTests(PyFlinkBlinkBatchTableTestCase):
             "sink2",
             CsvTableSink(field_names, field_types, "path2"))
 
-        t_env.sql_update("insert into sink1 select * from %s where a > 100" % source)
-        t_env.sql_update("insert into sink2 select * from %s where a < 100" % source)
+        stmt_set = t_env.create_statement_set()
+        stmt_set.add_insert_sql("insert into sink1 select * from %s where a > 100" % source)
+        stmt_set.add_insert_sql("insert into sink2 select * from %s where a < 100" % source)
 
-        actual = t_env.explain(extended=True)
+        actual = stmt_set.explain(ExplainDetail.ESTIMATED_COST, ExplainDetail.CHANGELOG_MODE)
         self.assertIsInstance(actual, str)
 
     def test_register_java_function(self):
diff --git a/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java b/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java
index b9dfd9e..4acdb0a 100644
--- a/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java
+++ b/flink-python/src/test/java/org/apache/flink/client/python/PythonFunctionFactoryTest.java
@@ -86,40 +86,40 @@ public class PythonFunctionFactoryTest {
 
 	public static void testPythonFunctionFactory() {
 		// flink catalog
-		flinkTableEnv.sqlUpdate("create function func1 as 'test1.func1' language python");
+		flinkTableEnv.executeSql("create function func1 as 'test1.func1' language python");
 		verifyPlan(flinkSourceTable.select(call("func1", $("str"))), flinkTableEnv);
 
 		// flink catalog
-		flinkTableEnv.sqlUpdate("alter function func1 as 'test1.func1' language python");
+		flinkTableEnv.executeSql("alter function func1 as 'test1.func1' language python");
 		verifyPlan(flinkSourceTable.select(call("func1", $("str"))), flinkTableEnv);
 
 		// flink temporary catalog
-		flinkTableEnv.sqlUpdate("create temporary function func1 as 'test1.func1' language python");
+		flinkTableEnv.executeSql("create temporary function func1 as 'test1.func1' language python");
 		verifyPlan(flinkSourceTable.select(call("func1", $("str"))), flinkTableEnv);
 
 		// flink temporary system
-		flinkTableEnv.sqlUpdate("create temporary system function func1 as 'test1.func1' language python");
+		flinkTableEnv.executeSql("create temporary system function func1 as 'test1.func1' language python");
 		verifyPlan(flinkSourceTable.select(call("func1", $("str"))), flinkTableEnv);
 
 		// blink catalog
-		blinkTableEnv.sqlUpdate("create function func1 as 'test1.func1' language python");
+		blinkTableEnv.executeSql("create function func1 as 'test1.func1' language python");
 		verifyPlan(blinkSourceTable.select(call("func1", $("str"))), blinkTableEnv);
 
 		// blink catalog
-		blinkTableEnv.sqlUpdate("alter function func1 as 'test1.func1' language python");
+		blinkTableEnv.executeSql("alter function func1 as 'test1.func1' language python");
 		verifyPlan(blinkSourceTable.select(call("func1", $("str"))), blinkTableEnv);
 
 		// blink temporary catalog
-		blinkTableEnv.sqlUpdate("create temporary function func1 as 'test1.func1' language python");
+		blinkTableEnv.executeSql("create temporary function func1 as 'test1.func1' language python");
 		verifyPlan(blinkSourceTable.select(call("func1", $("str"))), blinkTableEnv);
 
 		// blink temporary system
-		blinkTableEnv.sqlUpdate("create temporary system function func1 as 'test1.func1' language python");
+		blinkTableEnv.executeSql("create temporary system function func1 as 'test1.func1' language python");
 		verifyPlan(blinkSourceTable.select(call("func1", $("str"))), blinkTableEnv);
 	}
 
 	private static void verifyPlan(Table table, TableEnvironment tableEnvironment) {
-		String plan = tableEnvironment.explain(table);
+		String plan = table.explain();
 		String expected = "PythonCalc(select=[func1(f0) AS _c0])";
 		if (!plan.contains(expected)) {
 			throw new AssertionError(String.format("This plan does not contains \"%s\":\n%s", expected, plan));
diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
index 2453044..ac1af6c 100644
--- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
+++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/ExecutionContext.java
@@ -693,7 +693,7 @@ public class ExecutionContext<ClusterID> {
 
 	private void registerTemporalTable(TemporalTableEntry temporalTableEntry) {
 		try {
-			final Table table = tableEnv.scan(temporalTableEntry.getHistoryTable());
+			final Table table = tableEnv.from(temporalTableEntry.getHistoryTable());
 			List<String> primaryKeyFields = temporalTableEntry.getPrimaryKeyFields();
 			if (primaryKeyFields.size() > 1) {
 				throw new ValidationException("Temporal tables over a composite primary key are not supported yet.");
diff --git a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java
index 9a4fdcb..24563cc 100644
--- a/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java
+++ b/flink-table/flink-sql-client/src/main/java/org/apache/flink/table/client/gateway/local/LocalExecutor.java
@@ -77,6 +77,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
 import java.util.regex.Pattern;
 
 import static org.apache.flink.util.Preconditions.checkNotNull;
@@ -355,7 +356,7 @@ public class LocalExecutor implements Executor {
 		final ExecutionContext<?> context = getExecutionContext(sessionId);
 		final TableEnvironment tEnv = context.getTableEnvironment();
 		try {
-			context.wrapClassLoader(() -> tEnv.sqlUpdate(ddl));
+			context.wrapClassLoader(() -> tEnv.executeSql(ddl));
 		} catch (Exception e) {
 			throw new SqlExecutionException("Could not create a table from statement: " + ddl, e);
 		}
@@ -366,7 +367,7 @@ public class LocalExecutor implements Executor {
 		final ExecutionContext<?> context = getExecutionContext(sessionId);
 		final TableEnvironment tEnv = context.getTableEnvironment();
 		try {
-			context.wrapClassLoader(() -> tEnv.sqlUpdate(ddl));
+			context.wrapClassLoader(() -> tEnv.executeSql(ddl));
 		} catch (Exception e) {
 			throw new SqlExecutionException("Could not drop table from statement: " + ddl, e);
 		}
@@ -435,7 +436,7 @@ public class LocalExecutor implements Executor {
 		final ExecutionContext<?> context = getExecutionContext(sessionId);
 		final TableEnvironment tableEnv = context.getTableEnvironment();
 		try {
-			return context.wrapClassLoader(() -> tableEnv.scan(name).getSchema());
+			return context.wrapClassLoader(() -> tableEnv.from(name).getSchema());
 		} catch (Throwable t) {
 			// catch everything such that the query does not crash the executor
 			throw new SqlExecutionException("No table with this name could be found.", t);
@@ -449,7 +450,7 @@ public class LocalExecutor implements Executor {
 		// translate
 		try {
 			final Table table = createTable(context, tableEnv, statement);
-			return context.wrapClassLoader(() -> tableEnv.explain(table));
+			return context.wrapClassLoader((Supplier<String>) table::explain);
 		} catch (Throwable t) {
 			// catch everything such that the query does not crash the executor
 			throw new SqlExecutionException("Invalid SQL statement.", t);
@@ -613,7 +614,7 @@ public class LocalExecutor implements Executor {
 	private <C> ResultDescriptor executeQueryInternal(String sessionId, ExecutionContext<C> context, String query) {
 		// create table
 		final Table table = createTable(context, context.getTableEnvironment(), query);
-
+		// TODO refactor this after Table#execute support all kinds of changes
 		// initialize result
 		final DynamicResult<C> result = resultStore.createResult(
 				context.getEnvironment(),
@@ -695,6 +696,9 @@ public class LocalExecutor implements Executor {
 	private <C> void applyUpdate(ExecutionContext<C> context, String updateStatement) {
 		final TableEnvironment tableEnv = context.getTableEnvironment();
 		try {
+			// TODO replace sqlUpdate with executeSql
+			// This needs we do more refactor, because we can't set the flinkConfig in ExecutionContext
+			// into StreamExecutionEnvironment
 			context.wrapClassLoader(() -> tableEnv.sqlUpdate(updateStatement));
 		} catch (Throwable t) {
 			// catch everything such that the statement does not crash the executor
diff --git a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java
index d6e903a..871875e 100644
--- a/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java
+++ b/flink-table/flink-sql-client/src/test/java/org/apache/flink/table/client/gateway/local/ExecutionContextTest.java
@@ -242,7 +242,7 @@ public class ExecutionContextTest {
 
 		assertArrayEquals(
 			new String[]{"integerField", "stringField", "rowtimeField", "integerField0", "stringField0", "rowtimeField0"},
-			tableEnv.scan("TemporalTableUsage").getSchema().getFieldNames());
+			tableEnv.from("TemporalTableUsage").getSchema().getFieldNames());
 
 		// Please delete this test after removing registerTableSource in SQL-CLI.
 		TableSchema tableSchema = tableEnv.from("EnrichmentSource").getSchema();
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java
index 0ebcc69..1a5825a 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/Table.java
@@ -1008,7 +1008,10 @@ public interface Table {
 	 *
 	 * @param tablePath The path of the registered {@link TableSink} to which the {@link Table} is
 	 *        written.
+	 * @deprecated use {@link #executeInsert(String)} for single sink,
+	 *             use {@link TableEnvironment#createStatementSet()} for multiple sinks.
 	 */
+	@Deprecated
 	void insertInto(String tablePath);
 
 	/**
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java
index c9ac65e..9b6d53c 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/TableEnvironment.java
@@ -325,7 +325,9 @@ public interface TableEnvironment {
 	 * Creates a table from a table source.
 	 *
 	 * @param source table source used as table
+	 * @deprecated use {@link #createTemporaryView(String, Table)}.
 	 */
+	@Deprecated
 	Table fromTableSource(TableSource<?> source);
 
 	/**
@@ -676,7 +678,8 @@ public interface TableEnvironment {
 	 *        written. This is to ensure at least the name of the {@link TableSink} is provided.
 	 * @param sinkPathContinued The remaining part of the path of the registered {@link TableSink} to which the
 	 *        {@link Table} is written.
-	 * @deprecated use {@link #insertInto(String, Table)}
+	 * @deprecated use {@link Table#executeInsert(String)} for single sink,
+	 *             use {@link TableEnvironment#createStatementSet()} for multiple sinks.
 	 */
 	@Deprecated
 	void insertInto(Table table, String sinkPath, String... sinkPathContinued);
@@ -689,7 +692,10 @@ public interface TableEnvironment {
 	 *
 	 * @param targetPath The path of the registered {@link TableSink} to which the {@link Table} is written.
 	 * @param table The Table to write to the sink.
+	 * @deprecated use {@link Table#executeInsert(String)} for single sink,
+	 *             use {@link TableEnvironment#createStatementSet()} for multiple sinks.
 	 */
+	@Deprecated
 	void insertInto(String targetPath, Table table);
 
 	/**
@@ -820,7 +826,9 @@ public interface TableEnvironment {
 	 * the result of the given {@link Table}.
 	 *
 	 * @param table The table for which the AST and execution plan will be returned.
+	 * @deprecated use {@link Table#explain(ExplainDetail...)}.
 	 */
+	@Deprecated
 	String explain(Table table);
 
 	/**
@@ -830,7 +838,9 @@ public interface TableEnvironment {
 	 * @param table The table for which the AST and execution plan will be returned.
 	 * @param extended if the plan should contain additional properties,
 	 * e.g. estimated cost, traits
+	 * @deprecated use {@link Table#explain(ExplainDetail...)}.
 	 */
+	@Deprecated
 	String explain(Table table, boolean extended);
 
 	/**
@@ -839,7 +849,9 @@ public interface TableEnvironment {
 	 *
 	 * @param extended if the plan should contain additional properties,
 	 * e.g. estimated cost, traits
+	 * @deprecated use {@link StatementSet#explain(ExplainDetail...)}.
 	 */
+	@Deprecated
 	String explain(boolean extended);
 
 	/**
@@ -968,7 +980,10 @@ public interface TableEnvironment {
 	 * This code snippet creates a job to read data from Kafka source into a CSV sink.
 	 *
 	 * @param stmt The SQL statement to evaluate.
+	 * @deprecated use {@link #executeSql(String)} for single statement,
+	 *             use {@link TableEnvironment#createStatementSet()} for multiple DML statements.
 	 */
+	@Deprecated
 	void sqlUpdate(String stmt);
 
 	/**
@@ -1130,7 +1145,10 @@ public interface TableEnvironment {
 	 * @param jobName Desired name of the job
 	 * @return The result of the job execution, containing elapsed time and accumulators.
 	 * @throws Exception which occurs during job execution.
+	 * @deprecated use {@link #executeSql(String)} or {@link Table#executeInsert(String)} for single sink,
+	 *             use {@link #createStatementSet()} for multiple sinks.
 	 */
+	@Deprecated
 	JobExecutionResult execute(String jobName) throws Exception;
 
 	/**
diff --git a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
index de7bd96..4bae503 100644
--- a/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
+++ b/flink-table/flink-table-api-java/src/main/java/org/apache/flink/table/api/internal/TableEnvironmentImpl.java
@@ -677,7 +677,7 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
 	public TableResult executeInternal(List<ModifyOperation> operations) {
 		List<Transformation<?>> transformations = translate(operations);
 		List<String> sinkIdentifierNames = extractSinkIdentifierNames(operations);
-		String jobName = "insert_into_" + String.join(",", sinkIdentifierNames);
+		String jobName = "insert-into_" + String.join(",", sinkIdentifierNames);
 		Pipeline pipeline = execEnv.createPipeline(transformations, tableConfig, jobName);
 		try {
 			JobClient jobClient = execEnv.executeAsync(pipeline);
@@ -1044,18 +1044,34 @@ public class TableEnvironmentImpl implements TableEnvironmentInternal {
 
 	/**
 	 * extract sink identifier names from {@link ModifyOperation}s.
+	 *
+	 * <p>If there are multiple ModifyOperations have same name,
+	 * an index suffix will be added at the end of the name to ensure each name is unique.
 	 */
 	private List<String> extractSinkIdentifierNames(List<ModifyOperation> operations) {
 		List<String> tableNames = new ArrayList<>(operations.size());
+		Map<String, Integer> tableNameToCount = new HashMap<>();
 		for (ModifyOperation operation : operations) {
 			if (operation instanceof CatalogSinkModifyOperation) {
 				ObjectIdentifier identifier = ((CatalogSinkModifyOperation) operation).getTableIdentifier();
-				tableNames.add(identifier.asSummaryString());
+				String fullName = identifier.asSummaryString();
+				tableNames.add(fullName);
+				tableNameToCount.put(fullName, tableNameToCount.getOrDefault(fullName, 0) + 1);
 			} else {
 				throw new UnsupportedOperationException("Unsupported operation: " + operation);
 			}
 		}
-		return tableNames;
+		Map<String, Integer> tableNameToIndex = new HashMap<>();
+		return tableNames.stream().map(tableName -> {
+					if (tableNameToCount.get(tableName) == 1) {
+						return tableName;
+					} else {
+						Integer index = tableNameToIndex.getOrDefault(tableName, 0) + 1;
+						tableNameToIndex.put(tableName, index);
+						return tableName + "_" + index;
+					}
+				}
+		).collect(Collectors.toList());
 	}
 
 	/** Get catalog from catalogName or throw a ValidationException if the catalog not exists. */
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/BatchSelectTableSink.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/BatchSelectTableSink.java
index a1b5dea..adf08b0 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/BatchSelectTableSink.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/BatchSelectTableSink.java
@@ -56,7 +56,8 @@ public class BatchSelectTableSink implements StreamTableSink<Row>, SelectTableSi
 
 	@SuppressWarnings("unchecked")
 	public BatchSelectTableSink(TableSchema tableSchema) {
-		this.tableSchema = SelectTableSinkSchemaConverter.convert(tableSchema);
+		this.tableSchema = SelectTableSinkSchemaConverter.convertTimeAttributeToRegularTimestamp(
+				SelectTableSinkSchemaConverter.changeDefaultConversionClass(tableSchema));
 		this.accumulatorName = new AbstractID().toString();
 		this.typeSerializer = (TypeSerializer<Row>) TypeInfoDataTypeConverter
 				.fromDataTypeToTypeInfo(this.tableSchema.toRowDataType())
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/SelectTableSinkSchemaConverter.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/SelectTableSinkSchemaConverter.java
index ed571cb..7961c57 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/SelectTableSinkSchemaConverter.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/SelectTableSinkSchemaConverter.java
@@ -26,25 +26,37 @@ import org.apache.flink.table.types.logical.TimestampKind;
 import org.apache.flink.table.types.logical.TimestampType;
 
 /**
- * An utility class that changes to default conversion class
- * and converts time attributes (proc time / event time) to regular timestamps.
+ * An utility class that provides abilities to change {@link TableSchema}.
  */
 class SelectTableSinkSchemaConverter {
 
 	/**
-	 * Change to default conversion class and
-	 * convert time attributes (proc time / event time) to regular timestamps,
-	 * return a new {@link TableSchema}.
+	 * Change to default conversion class and build a new {@link TableSchema}.
 	 */
-	static TableSchema convert(TableSchema tableSchema) {
+	static TableSchema changeDefaultConversionClass(TableSchema tableSchema) {
 		DataType[] oldTypes = tableSchema.getFieldDataTypes();
-		String[] oldNames = tableSchema.getFieldNames();
+		String[] fieldNames = tableSchema.getFieldNames();
 
 		TableSchema.Builder builder = TableSchema.builder();
 		for (int i = 0; i < tableSchema.getFieldCount(); i++) {
-			// change to default conversion class
 			DataType fieldType = LogicalTypeDataTypeConverter.fromLogicalTypeToDataType(
 					LogicalTypeDataTypeConverter.fromDataTypeToLogicalType(oldTypes[i]));
+			builder.field(fieldNames[i], fieldType);
+		}
+		return builder.build();
+	}
+
+	/**
+	 * Convert time attributes (proc time / event time) to regular timestamp
+	 * and build a new {@link TableSchema}.
+	 */
+	static TableSchema convertTimeAttributeToRegularTimestamp(TableSchema tableSchema) {
+		DataType[] dataTypes = tableSchema.getFieldDataTypes();
+		String[] oldNames = tableSchema.getFieldNames();
+
+		TableSchema.Builder builder = TableSchema.builder();
+		for (int i = 0; i < tableSchema.getFieldCount(); i++) {
+			DataType fieldType = dataTypes[i];
 			String fieldName = oldNames[i];
 			if (fieldType.getLogicalType() instanceof TimestampType) {
 				TimestampType timestampType = (TimestampType) fieldType.getLogicalType();
diff --git a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/StreamSelectTableSink.java b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/StreamSelectTableSink.java
index a886db2..d0a1fa9 100644
--- a/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/StreamSelectTableSink.java
+++ b/flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/sinks/StreamSelectTableSink.java
@@ -52,7 +52,8 @@ public class StreamSelectTableSink implements AppendStreamTableSink<Row>, Select
 
 	@SuppressWarnings("unchecked")
 	public StreamSelectTableSink(TableSchema tableSchema) {
-		this.tableSchema = SelectTableSinkSchemaConverter.convert(tableSchema);
+		this.tableSchema = SelectTableSinkSchemaConverter.convertTimeAttributeToRegularTimestamp(
+				SelectTableSinkSchemaConverter.changeDefaultConversionClass(tableSchema));
 		this.typeSerializer = (TypeSerializer<Row>) TypeInfoDataTypeConverter
 				.fromDataTypeToTypeInfo(this.tableSchema.toRowDataType())
 				.createSerializer(new ExecutionConfig());
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogITCase.java
index ed50ef5..552c645 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogITCase.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/catalog/CatalogITCase.java
@@ -40,7 +40,7 @@ public class CatalogITCase {
 		TableEnvironment tableEnv = getTableEnvironment();
 		String ddl = String.format("create catalog %s with('type'='%s')", name, CATALOG_TYPE_VALUE_GENERIC_IN_MEMORY);
 
-		tableEnv.sqlUpdate(ddl);
+		tableEnv.executeSql(ddl);
 
 		assertTrue(tableEnv.getCatalog(name).isPresent());
 		assertTrue(tableEnv.getCatalog(name).get() instanceof GenericInMemoryCatalog);
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java
index c01ed29..319e942 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/FunctionITCase.java
@@ -25,6 +25,7 @@ import org.apache.flink.table.annotation.FunctionHint;
 import org.apache.flink.table.annotation.InputGroup;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.catalog.Catalog;
 import org.apache.flink.table.catalog.CatalogFunction;
@@ -73,10 +74,10 @@ public class FunctionITCase extends StreamingTestBase {
 	@Test
 	public void testCreateCatalogFunctionInDefaultCatalog() {
 		String ddl1 = "create function f1 as 'org.apache.flink.function.TestFunction'";
-		tEnv().sqlUpdate(ddl1);
+		tEnv().executeSql(ddl1);
 		assertTrue(Arrays.asList(tEnv().listFunctions()).contains("f1"));
 
-		tEnv().sqlUpdate("DROP FUNCTION IF EXISTS default_catalog.default_database.f1");
+		tEnv().executeSql("DROP FUNCTION IF EXISTS default_catalog.default_database.f1");
 		assertFalse(Arrays.asList(tEnv().listFunctions()).contains("f1"));
 	}
 
@@ -84,10 +85,10 @@ public class FunctionITCase extends StreamingTestBase {
 	public void testCreateFunctionWithFullPath() {
 		String ddl1 = "create function default_catalog.default_database.f2 as" +
 			" 'org.apache.flink.function.TestFunction'";
-		tEnv().sqlUpdate(ddl1);
+		tEnv().executeSql(ddl1);
 		assertTrue(Arrays.asList(tEnv().listFunctions()).contains("f2"));
 
-		tEnv().sqlUpdate("DROP FUNCTION IF EXISTS default_catalog.default_database.f2");
+		tEnv().executeSql("DROP FUNCTION IF EXISTS default_catalog.default_database.f2");
 		assertFalse(Arrays.asList(tEnv().listFunctions()).contains("f2"));
 	}
 
@@ -95,10 +96,10 @@ public class FunctionITCase extends StreamingTestBase {
 	public void testCreateFunctionWithoutCatalogIdentifier() {
 		String ddl1 = "create function default_database.f3 as" +
 			" 'org.apache.flink.function.TestFunction'";
-		tEnv().sqlUpdate(ddl1);
+		tEnv().executeSql(ddl1);
 		assertTrue(Arrays.asList(tEnv().listFunctions()).contains("f3"));
 
-		tEnv().sqlUpdate("DROP FUNCTION IF EXISTS default_catalog.default_database.f3");
+		tEnv().executeSql("DROP FUNCTION IF EXISTS default_catalog.default_database.f3");
 		assertFalse(Arrays.asList(tEnv().listFunctions()).contains("f3"));
 	}
 
@@ -107,7 +108,7 @@ public class FunctionITCase extends StreamingTestBase {
 		String ddl1 = "create function catalog1.database1.f3 as 'org.apache.flink.function.TestFunction'";
 
 		try {
-			tEnv().sqlUpdate(ddl1);
+			tEnv().executeSql(ddl1);
 		} catch (Exception e){
 			assertEquals("Catalog catalog1 does not exist", e.getMessage());
 		}
@@ -118,7 +119,7 @@ public class FunctionITCase extends StreamingTestBase {
 		String ddl1 = "create function default_catalog.database1.f3 as 'org.apache.flink.function.TestFunction'";
 
 		try {
-			tEnv().sqlUpdate(ddl1);
+			tEnv().executeSql(ddl1);
 		} catch (Exception e){
 			assertEquals(e.getMessage(), "Could not execute CREATE CATALOG FUNCTION:" +
 				" (catalogFunction: [Optional[This is a user-defined function]], identifier:" +
@@ -138,18 +139,18 @@ public class FunctionITCase extends StreamingTestBase {
 
 		String ddl4 = "drop temporary function if exists default_catalog.default_database.f4";
 
-		tEnv().sqlUpdate(ddl1);
+		tEnv().executeSql(ddl1);
 		assertTrue(Arrays.asList(tEnv().listFunctions()).contains("f4"));
 
-		tEnv().sqlUpdate(ddl2);
+		tEnv().executeSql(ddl2);
 		assertTrue(Arrays.asList(tEnv().listFunctions()).contains("f4"));
 
-		tEnv().sqlUpdate(ddl3);
+		tEnv().executeSql(ddl3);
 		assertFalse(Arrays.asList(tEnv().listFunctions()).contains("f4"));
 
-		tEnv().sqlUpdate(ddl1);
+		tEnv().executeSql(ddl1);
 		try {
-			tEnv().sqlUpdate(ddl1);
+			tEnv().executeSql(ddl1);
 		} catch (Exception e) {
 			assertTrue(e instanceof ValidationException);
 			assertEquals(e.getMessage(),
@@ -157,10 +158,10 @@ public class FunctionITCase extends StreamingTestBase {
 					" is already defined");
 		}
 
-		tEnv().sqlUpdate(ddl3);
-		tEnv().sqlUpdate(ddl4);
+		tEnv().executeSql(ddl3);
+		tEnv().executeSql(ddl4);
 		try {
-			tEnv().sqlUpdate(ddl3);
+			tEnv().executeSql(ddl3);
 		} catch (Exception e) {
 			assertTrue(e instanceof ValidationException);
 			assertEquals(e.getMessage(),
@@ -179,9 +180,9 @@ public class FunctionITCase extends StreamingTestBase {
 
 		String ddl3 = "drop temporary system function default_catalog.default_database.f5";
 
-		tEnv().sqlUpdate(ddl1);
-		tEnv().sqlUpdate(ddl2);
-		tEnv().sqlUpdate(ddl3);
+		tEnv().executeSql(ddl1);
+		tEnv().executeSql(ddl2);
+		tEnv().executeSql(ddl3);
 	}
 
 	@Test
@@ -192,11 +193,11 @@ public class FunctionITCase extends StreamingTestBase {
 		ObjectPath objectPath = new ObjectPath("default_database", "f3");
 		assertTrue(tEnv().getCatalog("default_catalog").isPresent());
 		Catalog catalog = tEnv().getCatalog("default_catalog").get();
-		tEnv().sqlUpdate(create);
+		tEnv().executeSql(create);
 		CatalogFunction beforeUpdate = catalog.getFunction(objectPath);
 		assertEquals("org.apache.flink.function.TestFunction", beforeUpdate.getClassName());
 
-		tEnv().sqlUpdate(alter);
+		tEnv().executeSql(alter);
 		CatalogFunction afterUpdate = catalog.getFunction(objectPath);
 		assertEquals("org.apache.flink.function.TestFunction2", afterUpdate.getClassName());
 	}
@@ -213,7 +214,7 @@ public class FunctionITCase extends StreamingTestBase {
 			"as 'org.apache.flink.function.TestFunction'";
 
 		try {
-			tEnv().sqlUpdate(alterUndefinedFunction);
+			tEnv().executeSql(alterUndefinedFunction);
 			fail();
 		} catch (Exception e){
 			assertEquals(e.getMessage(),
@@ -221,14 +222,14 @@ public class FunctionITCase extends StreamingTestBase {
 		}
 
 		try {
-			tEnv().sqlUpdate(alterFunctionInWrongCatalog);
+			tEnv().executeSql(alterFunctionInWrongCatalog);
 			fail();
 		} catch (Exception e) {
 			assertEquals("Catalog catalog1 does not exist", e.getMessage());
 		}
 
 		try {
-			tEnv().sqlUpdate(alterFunctionInWrongDB);
+			tEnv().executeSql(alterFunctionInWrongDB);
 			fail();
 		} catch (Exception e) {
 			assertEquals(e.getMessage(), "Function db1.f4 does not exist" +
@@ -242,7 +243,7 @@ public class FunctionITCase extends StreamingTestBase {
 			" as 'org.apache.flink.function.TestFunction'";
 
 		try {
-			tEnv().sqlUpdate(alterTemporary);
+			tEnv().executeSql(alterTemporary);
 			fail();
 		} catch (Exception e) {
 			assertEquals("Alter temporary catalog function is not supported", e.getMessage());
@@ -255,7 +256,7 @@ public class FunctionITCase extends StreamingTestBase {
 			" as 'org.apache.flink.function.TestFunction'";
 
 		try {
-			tEnv().sqlUpdate(alterTemporary);
+			tEnv().executeSql(alterTemporary);
 			fail();
 		} catch (Exception e) {
 			assertEquals("Alter temporary system function is not supported", e.getMessage());
@@ -271,7 +272,7 @@ public class FunctionITCase extends StreamingTestBase {
 		String dropFunctionInWrongDB = "DROP FUNCTION default_catalog.db1.f4";
 
 		try {
-			tEnv().sqlUpdate(dropUndefinedFunction);
+			tEnv().executeSql(dropUndefinedFunction);
 			fail();
 		} catch (Exception e){
 			assertEquals(e.getMessage(),
@@ -279,14 +280,14 @@ public class FunctionITCase extends StreamingTestBase {
 		}
 
 		try {
-			tEnv().sqlUpdate(dropFunctionInWrongCatalog);
+			tEnv().executeSql(dropFunctionInWrongCatalog);
 			fail();
 		} catch (Exception e) {
 			assertEquals("Catalog catalog1 does not exist", e.getMessage());
 		}
 
 		try {
-			tEnv().sqlUpdate(dropFunctionInWrongDB);
+			tEnv().executeSql(dropFunctionInWrongDB);
 			fail();
 		} catch (Exception e) {
 			assertEquals(e.getMessage(),
@@ -301,7 +302,7 @@ public class FunctionITCase extends StreamingTestBase {
 		String dropFunctionInWrongDB = "DROP TEMPORARY FUNCTION default_catalog.db1.f4";
 
 		try {
-			tEnv().sqlUpdate(dropUndefinedFunction);
+			tEnv().executeSql(dropUndefinedFunction);
 			fail();
 		} catch (Exception e){
 			assertEquals(e.getMessage(), "Temporary catalog function" +
@@ -309,7 +310,7 @@ public class FunctionITCase extends StreamingTestBase {
 		}
 
 		try {
-			tEnv().sqlUpdate(dropFunctionInWrongCatalog);
+			tEnv().executeSql(dropFunctionInWrongCatalog);
 			fail();
 		} catch (Exception e) {
 			assertEquals(e.getMessage(), "Temporary catalog function " +
@@ -317,7 +318,7 @@ public class FunctionITCase extends StreamingTestBase {
 		}
 
 		try {
-			tEnv().sqlUpdate(dropFunctionInWrongDB);
+			tEnv().executeSql(dropFunctionInWrongDB);
 			fail();
 		} catch (Exception e) {
 			assertEquals(e.getMessage(), "Temporary catalog function " +
@@ -332,24 +333,24 @@ public class FunctionITCase extends StreamingTestBase {
 
 		String dropNoCatalogDB = "drop temporary function f4";
 
-		tEnv().sqlUpdate(createNoCatalogDB);
-		tEnv().sqlUpdate(dropNoCatalogDB);
+		tEnv().executeSql(createNoCatalogDB);
+		tEnv().executeSql(dropNoCatalogDB);
 
 		String createNonExistsCatalog = "create temporary function catalog1.default_database.f4" +
 			" as '" + TEST_FUNCTION + "'";
 
 		String dropNonExistsCatalog = "drop temporary function catalog1.default_database.f4";
 
-		tEnv().sqlUpdate(createNonExistsCatalog);
-		tEnv().sqlUpdate(dropNonExistsCatalog);
+		tEnv().executeSql(createNonExistsCatalog);
+		tEnv().executeSql(dropNonExistsCatalog);
 
 		String createNonExistsDB = "create temporary function default_catalog.db1.f4" +
 			" as '" + TEST_FUNCTION + "'";
 
 		String dropNonExistsDB = "drop temporary function default_catalog.db1.f4";
 
-		tEnv().sqlUpdate(createNonExistsDB);
-		tEnv().sqlUpdate(dropNonExistsDB);
+		tEnv().executeSql(createNonExistsDB);
+		tEnv().executeSql(dropNonExistsDB);
 	}
 
 	@Test
@@ -360,12 +361,12 @@ public class FunctionITCase extends StreamingTestBase {
 
 		String ddl3 = "drop temporary system function if exists f5";
 
-		tEnv().sqlUpdate(ddl1);
-		tEnv().sqlUpdate(ddl2);
-		tEnv().sqlUpdate(ddl3);
+		tEnv().executeSql(ddl1);
+		tEnv().executeSql(ddl2);
+		tEnv().executeSql(ddl3);
 
 		try {
-			tEnv().sqlUpdate(ddl2);
+			tEnv().executeSql(ddl2);
 		} catch (Exception e) {
 			assertEquals(
 				e.getMessage(),
@@ -380,7 +381,7 @@ public class FunctionITCase extends StreamingTestBase {
 		String dropFunctionDDL = "drop function addOne";
 		testUserDefinedCatalogFunction(functionDDL);
 		// delete the function
-		tEnv().sqlUpdate(dropFunctionDDL);
+		tEnv().executeSql(dropFunctionDDL);
 	}
 
 	@Test
@@ -390,7 +391,7 @@ public class FunctionITCase extends StreamingTestBase {
 		String dropFunctionDDL = "drop temporary function addOne";
 		testUserDefinedCatalogFunction(functionDDL);
 		// delete the function
-		tEnv().sqlUpdate(dropFunctionDDL);
+		tEnv().executeSql(dropFunctionDDL);
 	}
 
 	@Test
@@ -400,7 +401,7 @@ public class FunctionITCase extends StreamingTestBase {
 		String dropFunctionDDL = "drop temporary system function addOne";
 		testUserDefinedCatalogFunction(functionDDL);
 		// delete the function
-		tEnv().sqlUpdate(dropFunctionDDL);
+		tEnv().executeSql(dropFunctionDDL);
 	}
 
 	/**
@@ -430,19 +431,19 @@ public class FunctionITCase extends StreamingTestBase {
 
 		String query = "select t1.a, t1.b, addOne(t1.a, 1) as c from t1";
 
-		tEnv().sqlUpdate(sourceDDL);
-		tEnv().sqlUpdate(sinkDDL);
-		tEnv().sqlUpdate(createFunctionDDL);
+		tEnv().executeSql(sourceDDL);
+		tEnv().executeSql(sinkDDL);
+		tEnv().executeSql(createFunctionDDL);
 		Table t2 = tEnv().sqlQuery(query);
-		tEnv().insertInto("t2", t2);
-		tEnv().execute("job1");
+		TableResult tableResult = t2.executeInsert("t2");
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 
 		Row[] result = TestCollectionTableFactory.RESULT().toArray(new Row[0]);
 		Row[] expected = sourceData.toArray(new Row[0]);
 		assertArrayEquals(expected, result);
 
-		tEnv().sqlUpdate("drop table t1");
-		tEnv().sqlUpdate("drop table t2");
+		tEnv().executeSql("drop table t1");
+		tEnv().executeSql("drop table t2");
 	}
 
 	@Test
@@ -462,11 +463,10 @@ public class FunctionITCase extends StreamingTestBase {
 		TestCollectionTableFactory.reset();
 		TestCollectionTableFactory.initData(sourceData);
 
-		tEnv().sqlUpdate("CREATE TABLE TestTable(i INT NOT NULL, b BIGINT NOT NULL, s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE TestTable(i INT NOT NULL, b BIGINT NOT NULL, s STRING) WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("PrimitiveScalarFunction", PrimitiveScalarFunction.class);
-		tEnv().sqlUpdate("INSERT INTO TestTable SELECT i, PrimitiveScalarFunction(i, b, s), s FROM TestTable");
-		tEnv().execute("Test Job");
+		execInsertSqlAndWaitResult("INSERT INTO TestTable SELECT i, PrimitiveScalarFunction(i, b, s), s FROM TestTable");
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData));
 	}
@@ -478,21 +478,20 @@ public class FunctionITCase extends StreamingTestBase {
 
 		TestCollectionTableFactory.reset();
 
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE TestTable(s1 STRING, s2 STRING, s3 STRING, s4 STRING, s5 STRING, s6 STRING) " +
 			"WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("ClassNameScalarFunction", ClassNameScalarFunction.class);
 		tEnv().createTemporarySystemFunction("ClassNameOrUnknownScalarFunction", ClassNameOrUnknownScalarFunction.class);
 		tEnv().createTemporarySystemFunction("WildcardClassNameScalarFunction", WildcardClassNameScalarFunction.class);
-		tEnv().sqlUpdate("INSERT INTO TestTable SELECT " +
+		execInsertSqlAndWaitResult("INSERT INTO TestTable SELECT " +
 			"ClassNameScalarFunction(NULL), " +
 			"ClassNameScalarFunction(CAST(NULL AS STRING)), " +
 			"ClassNameOrUnknownScalarFunction(NULL), " +
 			"ClassNameOrUnknownScalarFunction(CAST(NULL AS STRING)), " +
 			"WildcardClassNameScalarFunction(NULL), " +
 			"WildcardClassNameScalarFunction(CAST(NULL AS BOOLEAN))");
-		tEnv().execute("Test Job");
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData));
 	}
@@ -508,14 +507,13 @@ public class FunctionITCase extends StreamingTestBase {
 		TestCollectionTableFactory.reset();
 		TestCollectionTableFactory.initData(sourceData);
 
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE TestTable(i INT, r ROW<i INT, s STRING>) " +
 			"WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("RowScalarFunction", RowScalarFunction.class);
 		// the names of the function input and r differ
-		tEnv().sqlUpdate("INSERT INTO TestTable SELECT i, RowScalarFunction(r) FROM TestTable");
-		tEnv().execute("Test Job");
+		execInsertSqlAndWaitResult("INSERT INTO TestTable SELECT i, RowScalarFunction(r) FROM TestTable");
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sourceData));
 	}
@@ -563,10 +561,10 @@ public class FunctionITCase extends StreamingTestBase {
 			Object.class,
 			new KryoSerializer<>(Object.class, new ExecutionConfig()));
 
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE SourceTable(i INT, b BYTES) " +
 			"WITH ('connector' = 'COLLECTION')");
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE SinkTable(" +
 			"  i INT, " +
 			"  s1 STRING, " +
@@ -577,7 +575,7 @@ public class FunctionITCase extends StreamingTestBase {
 			"WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("ComplexScalarFunction", ComplexScalarFunction.class);
-		tEnv().sqlUpdate(
+		execInsertSqlAndWaitResult(
 			"INSERT INTO SinkTable " +
 			"SELECT " +
 			"  i, " +
@@ -586,7 +584,6 @@ public class FunctionITCase extends StreamingTestBase {
 			"  ComplexScalarFunction(), " +
 			"  ComplexScalarFunction(b) " +
 			"FROM SourceTable");
-		tEnv().execute("Test Job");
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData));
 	}
@@ -610,18 +607,17 @@ public class FunctionITCase extends StreamingTestBase {
 		TestCollectionTableFactory.reset();
 		TestCollectionTableFactory.initData(sourceData);
 
-		tEnv().sqlUpdate("CREATE TABLE SourceTable(i INT) WITH ('connector' = 'COLLECTION')");
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(i1 INT, i2 INT, i3 INT) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SourceTable(i INT) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(i1 INT, i2 INT, i3 INT) WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("CustomScalarFunction", CustomScalarFunction.class);
-		tEnv().sqlUpdate(
+		execInsertSqlAndWaitResult(
 			"INSERT INTO SinkTable " +
 			"SELECT " +
 			"  i, " +
 			"  CustomScalarFunction(i), " +
 			"  CustomScalarFunction(CAST(NULL AS INT), 5, i, i) " +
 			"FROM SourceTable");
-		tEnv().execute("Test Job");
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData));
 	}
@@ -668,13 +664,13 @@ public class FunctionITCase extends StreamingTestBase {
 			DayOfWeek.class,
 			new KryoSerializer<>(DayOfWeek.class, new ExecutionConfig()));
 
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE SourceTable(" +
 			"  i INT, " +
 			"  r " + rawType.asSerializableString() +
 			") " +
 			"WITH ('connector' = 'COLLECTION')");
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE SinkTable(" +
 			"  i INT, " +
 			"  s STRING, " +
@@ -683,7 +679,7 @@ public class FunctionITCase extends StreamingTestBase {
 			"WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("RawLiteralScalarFunction", RawLiteralScalarFunction.class);
-		tEnv().sqlUpdate(
+		execInsertSqlAndWaitResult(
 			"INSERT INTO SinkTable " +
 			"  (SELECT " +
 			"    i, " +
@@ -696,22 +692,19 @@ public class FunctionITCase extends StreamingTestBase {
 			"    RawLiteralScalarFunction(r, TRUE), " +
 			"    RawLiteralScalarFunction(r, FALSE) " +
 			"  FROM SourceTable)");
-		tEnv().execute("Test Job");
 
 		assertThat(TestCollectionTableFactory.getResult(), containsInAnyOrder(sinkData));
 	}
 
 	@Test
 	public void testInvalidCustomScalarFunction() {
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("CustomScalarFunction", CustomScalarFunction.class);
 		try {
-			tEnv().sqlUpdate(
+			execInsertSqlAndWaitResult(
 				"INSERT INTO SinkTable " +
 				"SELECT CustomScalarFunction('test')");
-			// trigger translation
-			tEnv().explain(false);
 			fail();
 		} catch (CodeGenException e) {
 			assertThat(
@@ -742,12 +735,11 @@ public class FunctionITCase extends StreamingTestBase {
 		TestCollectionTableFactory.reset();
 		TestCollectionTableFactory.initData(sourceData);
 
-		tEnv().sqlUpdate("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')");
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING, sa ARRAY<STRING> NOT NULL) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING, sa ARRAY<STRING> NOT NULL) WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("RowTableFunction", RowTableFunction.class);
-		tEnv().sqlUpdate("INSERT INTO SinkTable SELECT t.s, t.sa FROM SourceTable, LATERAL TABLE(RowTableFunction(s)) t");
-		tEnv().execute("Test Job");
+		execInsertSqlAndWaitResult("INSERT INTO SinkTable SELECT t.s, t.sa FROM SourceTable, LATERAL TABLE(RowTableFunction(s)) t");
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData));
 	}
@@ -762,32 +754,29 @@ public class FunctionITCase extends StreamingTestBase {
 
 		TestCollectionTableFactory.reset();
 
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("DynamicTableFunction", DynamicTableFunction.class);
-		tEnv().sqlUpdate(
+		execInsertSqlAndWaitResult(
 			"INSERT INTO SinkTable " +
 			"SELECT T1.s FROM TABLE(DynamicTableFunction('Test')) AS T1(s) " +
 			"UNION ALL " +
 			"SELECT CAST(T2.i AS STRING) FROM TABLE(DynamicTableFunction(42)) AS T2(i)" +
 			"UNION ALL " +
 			"SELECT CAST(T3.i AS STRING) FROM TABLE(DynamicTableFunction(CAST(NULL AS INT))) AS T3(i)");
-		tEnv().execute("Test Job");
 
 		assertThat(TestCollectionTableFactory.getResult(), containsInAnyOrder(sinkData));
 	}
 
 	@Test
 	public void testInvalidUseOfScalarFunction() {
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("PrimitiveScalarFunction", PrimitiveScalarFunction.class);
 		try {
-			tEnv().sqlUpdate(
+			tEnv().executeSql(
 				"INSERT INTO SinkTable " +
 				"SELECT * FROM TABLE(PrimitiveScalarFunction(1, 2, '3'))");
-			// trigger translation
-			tEnv().explain(false);
 			fail();
 		} catch (ValidationException e) {
 			assertThat(
@@ -800,14 +789,12 @@ public class FunctionITCase extends StreamingTestBase {
 
 	@Test
 	public void testInvalidUseOfSystemScalarFunction() {
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
 
 		try {
-			tEnv().sqlUpdate(
+			tEnv().explainSql(
 				"INSERT INTO SinkTable " +
 				"SELECT * FROM TABLE(MD5('3'))");
-			// trigger translation
-			tEnv().explain(false);
 			fail();
 		} catch (ValidationException e) {
 			assertThat(
@@ -820,11 +807,11 @@ public class FunctionITCase extends StreamingTestBase {
 
 	@Test
 	public void testInvalidUseOfTableFunction() {
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING) WITH ('connector' = 'COLLECTION')");
 
 		tEnv().createTemporarySystemFunction("RowTableFunction", RowTableFunction.class);
 		try {
-			tEnv().sqlUpdate(
+			tEnv().executeSql(
 				"INSERT INTO SinkTable " +
 				"SELECT RowTableFunction('test')");
 			fail();
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java
index 3e67bab..2fb79da 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/FunctionITCase.java
@@ -20,6 +20,8 @@ package org.apache.flink.table.planner.runtime.stream.table;
 
 import org.apache.flink.table.annotation.DataTypeHint;
 import org.apache.flink.table.annotation.FunctionHint;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.api.ValidationException;
 import org.apache.flink.table.functions.ScalarFunction;
 import org.apache.flink.table.functions.TableFunction;
@@ -64,18 +66,17 @@ public class FunctionITCase extends StreamingTestBase {
 		TestCollectionTableFactory.reset();
 		TestCollectionTableFactory.initData(sourceData);
 
-		tEnv().sqlUpdate("CREATE TABLE TestTable(a INT, b BIGINT, c BIGINT) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE TestTable(a INT, b BIGINT, c BIGINT) WITH ('connector' = 'COLLECTION')");
 
-		tEnv().from("TestTable")
+		Table table = tEnv().from("TestTable")
 			.select(
 				$("a"),
 				call(new SimpleScalarFunction(), $("a"), $("b")),
 				call(new SimpleScalarFunction(), $("a"), $("b"))
 					.plus(1)
 					.minus(call(new SimpleScalarFunction(), $("a"), $("b")))
-			)
-			.insertInto("TestTable");
-		tEnv().execute("Test Job");
+			);
+		execInsertTableAndWaitResult(table, "TestTable");
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData));
 	}
@@ -98,14 +99,14 @@ public class FunctionITCase extends StreamingTestBase {
 		TestCollectionTableFactory.reset();
 		TestCollectionTableFactory.initData(sourceData);
 
-		tEnv().sqlUpdate("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')");
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING, sa ARRAY<STRING>) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING, sa ARRAY<STRING>) WITH ('connector' = 'COLLECTION')");
 
-		tEnv().from("SourceTable")
+		TableResult tableResult = tEnv().from("SourceTable")
 			.joinLateral(call(new SimpleTableFunction(), $("s")).as("a", "b"))
 			.select($("a"), $("b"))
-			.insertInto("SinkTable");
-		tEnv().execute("Test Job");
+			.executeInsert("SinkTable");
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 
 		assertThat(TestCollectionTableFactory.getResult(), equalTo(sinkData));
 	}
@@ -116,14 +117,14 @@ public class FunctionITCase extends StreamingTestBase {
 		thrown.expectMessage("Currently, only table functions can emit rows.");
 
 		TestCollectionTableFactory.reset();
-		tEnv().sqlUpdate("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')");
-		tEnv().sqlUpdate("CREATE TABLE SinkTable(s STRING, sa ARRAY<STRING>) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SourceTable(s STRING) WITH ('connector' = 'COLLECTION')");
+		tEnv().executeSql("CREATE TABLE SinkTable(s STRING, sa ARRAY<STRING>) WITH ('connector' = 'COLLECTION')");
 
-		tEnv().from("SourceTable")
+		TableResult tableResult = tEnv().from("SourceTable")
 			.joinLateral(call(new RowScalarFunction(), $("s")).as("a", "b"))
 			.select($("a"), $("b"))
-			.insertInto("SinkTable");
-		tEnv().execute("Test Job");
+			.executeInsert("SinkTable");
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 	}
 
 	// --------------------------------------------------------------------------------------------
diff --git a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/ValuesITCase.java b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/ValuesITCase.java
index 80a1aaf..66b894a 100644
--- a/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/ValuesITCase.java
+++ b/flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/runtime/stream/table/ValuesITCase.java
@@ -22,9 +22,11 @@ import org.apache.flink.table.annotation.DataTypeHint;
 import org.apache.flink.table.annotation.FunctionHint;
 import org.apache.flink.table.api.DataTypes;
 import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableResult;
 import org.apache.flink.table.functions.ScalarFunction;
 import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory;
 import org.apache.flink.table.planner.runtime.utils.StreamingTestBase;
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil;
 import org.apache.flink.table.types.DataType;
 import org.apache.flink.types.Row;
 
@@ -103,15 +105,15 @@ public class ValuesITCase extends StreamingTestBase {
 		);
 
 		TestCollectionTableFactory.reset();
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE SinkTable(" +
 				"a DECIMAL(10, 2) NOT NULL, " +
 				"b CHAR(4) NOT NULL," +
 				"c TIMESTAMP(4) NOT NULL," +
 				"`row` ROW<a DECIMAL(10, 3) NOT NULL, b BINARY(2), c CHAR(5) NOT NULL, d ARRAY<DECIMAL(10, 2)>>) " +
 				"WITH ('connector' = 'COLLECTION')");
-		t.insertInto("SinkTable");
-		tEnv().execute("");
+		TableResult tableResult = t.executeInsert("SinkTable");
+		tableResult.getJobClient().get().getJobExecutionResult(Thread.currentThread().getContextClassLoader()).get();
 
 		List<Row> expected = Arrays.asList(
 			Row.of(
@@ -206,7 +208,7 @@ public class ValuesITCase extends StreamingTestBase {
 		Table t = tEnv().fromValues(data);
 
 		TestCollectionTableFactory.reset();
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE SinkTable(" +
 				"f0 TINYINT, " +
 				"f1 SMALLINT, " +
@@ -242,8 +244,7 @@ public class ValuesITCase extends StreamingTestBase {
 				"   `f16` ARRAY<DECIMAL(2, 1)>, " +
 				"   `f17` MAP<CHAR(1), DECIMAL(2, 1)>>) " +
 				"WITH ('connector' = 'COLLECTION')");
-		t.insertInto("SinkTable");
-		tEnv().execute("");
+		execInsertTableAndWaitResult(t, "SinkTable");
 
 		List<Row> actual = TestCollectionTableFactory.getResult();
 		assertThat(
@@ -297,10 +298,9 @@ public class ValuesITCase extends StreamingTestBase {
 			.select(call("func", withColumns(range("f0", "f15"))));
 
 		TestCollectionTableFactory.reset();
-		tEnv().sqlUpdate(
+		tEnv().executeSql(
 			"CREATE TABLE SinkTable(str STRING) WITH ('connector' = 'COLLECTION')");
-		t.insertInto("SinkTable");
-		tEnv().execute("");
+		TableEnvUtil.execInsertTableAndWaitResult(t, "SinkTable");
 
 		List<Row> actual = TestCollectionTableFactory.getResult();
 		List<Row> expected = Arrays.asList(
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala
index 505e9a9..d3c9034 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentITCase.scala
@@ -26,7 +26,7 @@ import org.apache.flink.table.api.internal.TableEnvironmentImpl
 import org.apache.flink.table.api.java.StreamTableEnvironment
 import org.apache.flink.table.api.scala.{StreamTableEnvironment => ScalaStreamTableEnvironment, _}
 import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory
-import org.apache.flink.table.planner.runtime.utils.TestingAppendSink
+import org.apache.flink.table.planner.runtime.utils.{TableEnvUtil, TestingAppendSink}
 import org.apache.flink.table.planner.utils.TableTestUtil.{readFromResource, replaceStageId}
 import org.apache.flink.table.planner.utils.{TableTestUtil, TestTableSourceSinks, TestTableSourceWithTime}
 import org.apache.flink.types.Row
@@ -293,7 +293,7 @@ class TableEnvironmentITCase(tableEnvName: String, isStreaming: Boolean) extends
     }
 
     val sinkPath = _tempFolder.newFolder().toString
-    tEnv.sqlUpdate(
+    tEnv.executeSql(
       s"""
          |create table MySink (
          |  first string
@@ -536,6 +536,41 @@ class TableEnvironmentITCase(tableEnvName: String, isStreaming: Boolean) extends
   }
 
   @Test
+  def testStatementSetWithSameSinkTableNames(): Unit = {
+    if(isStreaming) {
+      // Streaming mode not support overwrite for FileSystemTableSink.
+      return
+    }
+    val sinkPath = _tempFolder.newFolder().toString
+    tEnv.executeSql(
+      s"""
+         |create table MySink (
+         |  first string
+         |) with (
+         |  'connector' = 'filesystem',
+         |  'path' = '$sinkPath',
+         |  'format' = 'testcsv'
+         |)
+       """.stripMargin
+    )
+
+    val stmtSet = tEnv.createStatementSet()
+    stmtSet.addInsert("MySink", tEnv.sqlQuery("select first from MyTable"), true)
+    stmtSet.addInsertSql("insert overwrite MySink select last from MyTable")
+
+    val tableResult = stmtSet.execute()
+    // wait job finished
+    tableResult.getJobClient.get()
+      .getJobExecutionResult(Thread.currentThread().getContextClassLoader)
+      .get()
+    // only check the schema
+    checkInsertTableResult(
+      tableResult,
+      "default_catalog.default_database.MySink_1",
+      "default_catalog.default_database.MySink_2")
+  }
+
+  @Test
   def testExecuteSelect(): Unit = {
     val query =
       """
@@ -607,22 +642,20 @@ class TableEnvironmentITCase(tableEnvName: String, isStreaming: Boolean) extends
   def testClearOperation(): Unit = {
     TestCollectionTableFactory.reset()
     val tableEnv = TableEnvironmentImpl.create(settings)
-    tableEnv.sqlUpdate("create table dest1(x map<int,bigint>) with('connector' = 'COLLECTION')")
-    tableEnv.sqlUpdate("create table dest2(x int) with('connector' = 'COLLECTION')")
-    tableEnv.sqlUpdate("create table src(x int) with('connector' = 'COLLECTION')")
+    tableEnv.executeSql("create table dest1(x map<int,bigint>) with('connector' = 'COLLECTION')")
+    tableEnv.executeSql("create table dest2(x int) with('connector' = 'COLLECTION')")
+    tableEnv.executeSql("create table src(x int) with('connector' = 'COLLECTION')")
 
     try {
       // it would fail due to query and sink type mismatch
-      tableEnv.sqlUpdate("insert into dest1 select count(*) from src")
-      tableEnv.execute("insert dest1")
+      tableEnv.executeSql("insert into dest1 select count(*) from src")
       Assert.fail("insert is expected to fail due to type mismatch")
     } catch {
       case _: Exception => //expected
     }
 
-    tableEnv.sqlUpdate("drop table dest1")
-    tableEnv.sqlUpdate("insert into dest2 select x from src")
-    tableEnv.execute("insert dest2")
+    tableEnv.executeSql("drop table dest1")
+    TableEnvUtil.execInsertSqlAndWaitResult(tableEnv, "insert into dest2 select x from src")
   }
 
   def getPersonData: List[(String, Int, Double, String)] = {
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
index a890d7c..457e2c6 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/api/TableEnvironmentTest.scala
@@ -56,14 +56,14 @@ class TableEnvironmentTest {
   def testScanNonExistTable(): Unit = {
     thrown.expect(classOf[ValidationException])
     thrown.expectMessage("Table `MyTable` was not found")
-    tableEnv.scan("MyTable")
+    tableEnv.from("MyTable")
   }
 
   @Test
   def testRegisterDataStream(): Unit = {
     val table = env.fromElements[(Int, Long, String, Boolean)]().toTable(tableEnv, 'a, 'b, 'c, 'd)
     tableEnv.registerTable("MyTable", table)
-    val scanTable = tableEnv.scan("MyTable")
+    val scanTable = tableEnv.from("MyTable")
     val relNode = TableTestUtil.toRelNode(scanTable)
     val actual = RelOptUtil.toString(relNode)
     val expected = "LogicalTableScan(table=[[default_catalog, default_database, MyTable]])\n"
@@ -542,10 +542,10 @@ class TableEnvironmentTest {
 
     assert(tableEnv.listTables().sameElements(Array[String]("T1", "T2", "T3")))
 
-    tableEnv.sqlUpdate("DROP VIEW default_catalog.default_database.T2")
+    tableEnv.executeSql("DROP VIEW default_catalog.default_database.T2")
     assert(tableEnv.listTables().sameElements(Array[String]("T1", "T3")))
 
-    tableEnv.sqlUpdate("DROP VIEW default_catalog.default_database.T3")
+    tableEnv.executeSql("DROP VIEW default_catalog.default_database.T3")
     assert(tableEnv.listTables().sameElements(Array[String]("T1")))
   }
 
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 4ae9b88..bc0e840 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
@@ -19,18 +19,25 @@
 package org.apache.flink.table.api.batch
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.utils.TableTestBase
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
 
-import org.junit.{Before, Test}
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
+import org.junit.{Before, Test}
 
 @RunWith(classOf[Parameterized])
 class ExplainTest(extended: Boolean) extends TableTestBase {
 
+  private val extraDetails = if (extended) {
+    Array(ExplainDetail.CHANGELOG_MODE, ExplainDetail.ESTIMATED_COST)
+  } else {
+    Array.empty[ExplainDetail]
+  }
+
   private val util = batchTestUtil()
   util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
   util.addDataStream[(Int, Long, String)]("MyTable1", 'a, 'b, 'c)
@@ -48,22 +55,22 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
 
   @Test
   def testExplainWithTableSourceScan(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable", extended)
+    util.verifyExplain("SELECT * FROM MyTable", extraDetails: _*)
   }
 
   @Test
   def testExplainWithDataStreamScan(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1", extended)
+    util.verifyExplain("SELECT * FROM MyTable1", extraDetails: _*)
   }
 
   @Test
   def testExplainWithFilter(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extended)
+    util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extraDetails: _*)
   }
 
   @Test
   def testExplainWithAgg(): Unit = {
-    util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extended)
+    util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extraDetails: _*)
   }
 
   @Test
@@ -71,41 +78,43 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
     // TODO support other join operators when them are supported
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_DISABLED_OPERATORS, "HashJoin, NestedLoopJoin")
-    util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extended)
+    util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extraDetails: _*)
   }
 
   @Test
   def testExplainWithUnion(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extended)
+    util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extraDetails: _*)
   }
 
   @Test
   def testExplainWithSort(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extended)
+    util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extraDetails: _*)
   }
 
   @Test
   def testExplainWithSingleSink(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT * FROM MyTable1 WHERE a > 10")
     val sink = util.createCollectTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table, sink, "sink")
-    util.verifyExplain(extended)
+    util.verifyExplainInsert(table, sink, "sink", extraDetails: _*)
   }
 
   @Test
   def testExplainWithMultiSinks(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable1 GROUP BY a")
     util.tableEnv.registerTable("TempTable", table)
 
     val table1 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt > 10")
     val sink1 = util.createCollectTableSink(Array("a", "cnt"), Array(INT, LONG))
-    util.writeToSink(table1, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt < 10")
     val sink2 = util.createCollectTableSink(Array("a", "cnt"), Array(INT, LONG))
-    util.writeToSink(table2, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table2)
 
-    util.verifyExplain(extended)
+    util.verifyExplain(stmtSet, extraDetails: _*)
   }
 
 }
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 198fb0f..8c317af 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
@@ -19,20 +19,27 @@
 package org.apache.flink.table.api.stream
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.utils.TableTestBase
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
 
-import org.junit.{Before, Test}
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
+import org.junit.{Before, Test}
 
 import java.sql.Timestamp
 
 @RunWith(classOf[Parameterized])
 class ExplainTest(extended: Boolean) extends TableTestBase {
 
+  private val extraDetails = if (extended) {
+    Array(ExplainDetail.CHANGELOG_MODE, ExplainDetail.ESTIMATED_COST)
+  } else {
+    Array.empty[ExplainDetail]
+  }
+
   private val util = streamTestUtil()
   util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
   util.addDataStream[(Int, Long, String)]("MyTable1", 'a, 'b, 'c)
@@ -50,71 +57,74 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
 
   @Test
   def testExplainTableSourceScan(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable", extended)
+    util.verifyExplain("SELECT * FROM MyTable", extraDetails:_*)
   }
 
   @Test
   def testExplainDataStreamScan(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1", extended)
+    util.verifyExplain("SELECT * FROM MyTable1", extraDetails:_*)
   }
 
   @Test
   def testExplainWithFilter(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extended)
+    util.verifyExplain("SELECT * FROM MyTable1 WHERE mod(a, 2) = 0", extraDetails:_*)
   }
 
   @Test
   def testExplainWithAgg(): Unit = {
-    util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extended)
+    util.verifyExplain("SELECT COUNT(*) FROM MyTable1 GROUP BY a", extraDetails:_*)
   }
 
   @Test
   def testExplainWithJoin(): Unit = {
-    util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extended)
+    util.verifyExplain("SELECT a, b, c, e, f FROM MyTable1, MyTable2 WHERE a = d", extraDetails:_*)
   }
 
   @Test
   def testExplainWithUnion(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extended)
+    util.verifyExplain("SELECT * FROM MyTable1 UNION ALL SELECT * FROM MyTable2", extraDetails:_*)
   }
 
   @Test
   def testExplainWithSort(): Unit = {
-    util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extended)
+    util.verifyExplain("SELECT * FROM MyTable1 ORDER BY a LIMIT 5", extraDetails:_*)
   }
 
   @Test
   def testExplainWithSingleSink(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT * FROM MyTable1 WHERE a > 10")
     val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table, appendSink, "appendSink")
-    util.verifyExplain(extended)
+    util.verifyExplainInsert(table, appendSink, "appendSink", extraDetails: _*)
   }
 
   @Test
   def testExplainWithMultiSinks(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable1 GROUP BY a")
     util.tableEnv.registerTable("TempTable", table)
 
     val table1 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt > 10")
     val upsertSink1 = util.createUpsertTableSink(Array(0), Array("a", "cnt"), Array(INT, LONG))
-    util.writeToSink(table1, upsertSink1, "upsertSink1")
+    util.tableEnv.registerTableSink("upsertSink1", upsertSink1)
+    stmtSet.addInsert("upsertSink1", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT * FROM TempTable WHERE cnt < 10")
     val upsertSink2 = util.createUpsertTableSink(Array(0), Array("a", "cnt"), Array(INT, LONG))
-    util.writeToSink(table2, upsertSink2, "upsertSink2")
+    util.tableEnv.registerTableSink("upsertSink2", upsertSink2)
+    stmtSet.addInsert("upsertSink2", table2)
 
-    util.verifyExplain(extended)
+    util.verifyExplain(stmtSet, extraDetails: _*)
   }
 
   @Test
   def testMiniBatchIntervalInfer(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     // Test emit latency propagate among RelNodeBlocks
     util.addDataStream[(Int, String, Timestamp)]("T1", 'id1, 'text, 'rowtime.rowtime)
     util.addDataStream[(Int, String, Int, String, Long, Timestamp)](
       "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.addTableWithWatermark("T3", util.tableEnv.from("T1"), "rowtime", 0)
+    util.addTableWithWatermark("T4", util.tableEnv.from("T2"), "rowtime", 0)
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ENABLED, true)
     util.tableEnv.getConfig.getConfiguration.setString(
@@ -136,7 +146,8 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
         |GROUP BY id1, TUMBLE(ts, INTERVAL '8' SECOND)
       """.stripMargin)
     val appendSink1 = util.createAppendTableSink(Array("a", "b"), Array(INT, STRING))
-    util.writeToSink(table1, appendSink1, "appendSink1")
+    util.tableEnv.registerTableSink("appendSink1", appendSink1)
+    stmtSet.addInsert("appendSink1", table1)
 
     val table2 = util.tableEnv.sqlQuery(
       """
@@ -145,9 +156,10 @@ class ExplainTest(extended: Boolean) extends TableTestBase {
         |GROUP BY id1, HOP(ts, INTERVAL '12' SECOND, INTERVAL '6' SECOND)
       """.stripMargin)
     val appendSink2 = util.createAppendTableSink(Array("a", "b"), Array(INT, STRING))
-    util.writeToSink(table2, appendSink2, "appendSink2")
+    util.tableEnv.registerTableSink("appendSink2", appendSink2)
+    stmtSet.addInsert("appendSink2", table2)
 
-    util.verifyExplain(extended)
+    util.verifyExplain(stmtSet, extraDetails: _*)
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala
index c5e6ba3..45db348 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableITCase.scala
@@ -25,19 +25,22 @@ import org.apache.flink.table.catalog.{CatalogDatabaseImpl, CatalogFunctionImpl,
 import org.apache.flink.table.planner.expressions.utils.Func0
 import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory
 import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.JavaFunc0
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil.{execInsertSqlAndWaitResult, execInsertTableAndWaitResult}
 import org.apache.flink.table.planner.utils.DateTimeTestUtil.localDateTime
 import org.apache.flink.test.util.AbstractTestBase
 import org.apache.flink.types.Row
 import org.apache.flink.util.FileUtils
+
 import org.junit.Assert.{assertEquals, fail}
 import org.junit.rules.ExpectedException
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 import org.junit.{Before, Rule, Test}
+
 import java.io.File
-import java.util
 import java.math.{BigDecimal => JBigDecimal}
 import java.net.URI
+import java.util
 
 import scala.collection.JavaConversions._
 
@@ -94,10 +97,6 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     row
   }
 
-  def execJob(name: String) = {
-    tableEnv.execute(name)
-  }
-
   //~ Tests ------------------------------------------------------------------
 
   private def testUdf(funcPrefix: String): Unit = {
@@ -110,9 +109,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |  'is-bounded' = '$isStreamingMode'
         |)
       """.stripMargin
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(s"insert into sinkT select ${funcPrefix}myfunc(cast(1 as bigint))")
-    tableEnv.execute("")
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(
+      tableEnv, s"insert into sinkT select ${funcPrefix}myfunc(cast(1 as bigint))")
     assertEquals(Seq(toRow(2L)), TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -178,10 +177,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select t1.a, t1.b, (t1.a + 1) as c , d from t1
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(sourceData.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -213,7 +211,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
        |  'format.line-delimiter' = '#'
        |)
      """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
+    tableEnv.executeSql(sourceDDL)
 
     val sinkFilePath = getTempFilePath("csv-order-sink")
     val sinkDDL =
@@ -230,7 +228,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |  'format.field-delimiter' = ','
         |)
       """.stripMargin
-    tableEnv.sqlUpdate(sinkDDL)
+    tableEnv.executeSql(sinkDDL)
 
     val query =
       """
@@ -243,8 +241,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |FROM T1
         |GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)
       """.stripMargin
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    execInsertSqlAndWaitResult(tableEnv, query)
 
     val expected =
       "2019-12-12 00:00:05.0,2019-12-12 00:00:04.004001,3,50.00\n" +
@@ -279,7 +276,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
          |  'format.field-delimiter' = ','
          |)
      """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
+    tableEnv.executeSql(sourceDDL)
 
     val sinkFilePath = getTempFilePath("csv-order-sink")
     val sinkDDL =
@@ -295,7 +292,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
          |  'format.type' = 'csv'
          |)
       """.stripMargin
-    tableEnv.sqlUpdate(sinkDDL)
+    tableEnv.executeSql(sinkDDL)
 
     val query =
       """
@@ -308,8 +305,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |FROM T1 /*+ OPTIONS('format.line-delimiter' = '#') */
         |GROUP BY TUMBLE(ts, INTERVAL '5' SECOND)
       """.stripMargin
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    execInsertSqlAndWaitResult(tableEnv, query)
 
     val expected =
       "2019-12-12 00:00:05.0|2019-12-12 00:00:04.004001|3|50.00\n" +
@@ -359,10 +355,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select t1.a, t1.b, t1.c from t1
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -409,10 +404,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select t1.c, t1.a, t1.b from t1
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -458,10 +452,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select t1.a, t1.b, t1.c from t1
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -510,10 +503,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select t1.a, t1.`time`, t1.c, t1.d from t1
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -559,10 +551,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select t1.a, t1.c from t1
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -601,13 +592,12 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select t1.a, t1.b from t1
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
     expectedEx.expect(classOf[ValidationException])
     expectedEx.expectMessage("Field types of query result and registered TableSink "
       + "default_catalog.default_database.t2 do not match.")
-    execJob("testJob")
+    tableEnv.executeSql(query)
   }
 
   @Test
@@ -656,10 +646,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |  join t1 b
         |  on a.a = b.b
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -706,10 +695,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |insert into t2
         |select sum(a), t1.b from t1 group by t1.b
       """.stripMargin
-    tableEnv.sqlUpdate(sourceDDL)
-    tableEnv.sqlUpdate(sinkDDL)
-    tableEnv.sqlUpdate(query)
-    execJob("testJob")
+    tableEnv.executeSql(sourceDDL)
+    tableEnv.executeSql(sinkDDL)
+    execInsertSqlAndWaitResult(tableEnv, query)
     assertEquals(expected.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -778,9 +766,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     TestCollectionTableFactory.initData(sourceData)
 
     val query = "SELECT a, b, d FROM T1"
-
-    tableEnv.sqlQuery(query).insertInto("T2")
-    execJob("testJob")
+    execInsertTableAndWaitResult(tableEnv.sqlQuery(query), "T2")
     // temporary table T1 masks permanent table T1
     assertEquals(temporaryData.sorted, TestCollectionTableFactory.RESULT.sorted)
 
@@ -792,8 +778,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |DROP TEMPORARY TABLE IF EXISTS T1
       """.stripMargin
     tableEnv.executeSql(dropTemporaryTable)
-    tableEnv.sqlQuery(query).insertInto("T2")
-    execJob("testJob")
+    execInsertTableAndWaitResult(tableEnv.sqlQuery(query), "T2")
     // now we only have permanent view T1
     assertEquals(permanentData.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
@@ -820,10 +805,10 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |)
       """.stripMargin
 
-    tableEnv.sqlUpdate(ddl1)
-    tableEnv.sqlUpdate(ddl2)
+    tableEnv.executeSql(ddl1)
+    tableEnv.executeSql(ddl2)
     assert(tableEnv.listTables().sameElements(Array[String]("t1", "t2")))
-    tableEnv.sqlUpdate("DROP TABLE default_catalog.default_database.t2")
+    tableEnv.executeSql("DROP TABLE default_catalog.default_database.t2")
     assert(tableEnv.listTables().sameElements(Array("t1")))
   }
 
@@ -849,11 +834,11 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |)
       """.stripMargin
 
-    tableEnv.sqlUpdate(ddl1)
-    tableEnv.sqlUpdate(ddl2)
+    tableEnv.executeSql(ddl1)
+    tableEnv.executeSql(ddl2)
     assert(tableEnv.listTables().sameElements(Array[String]("t1", "t2")))
-    tableEnv.sqlUpdate("DROP TABLE default_database.t2")
-    tableEnv.sqlUpdate("DROP TABLE t1")
+    tableEnv.executeSql("DROP TABLE default_database.t2")
+    tableEnv.executeSql("DROP TABLE t1")
     assert(tableEnv.listTables().isEmpty)
   }
 
@@ -870,9 +855,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |)
       """.stripMargin
 
-    tableEnv.sqlUpdate(ddl1)
+    tableEnv.executeSql(ddl1)
     assert(tableEnv.listTables().sameElements(Array[String]("t1")))
-    tableEnv.sqlUpdate("DROP TABLE catalog1.database1.t1")
+    tableEnv.executeSql("DROP TABLE catalog1.database1.t1")
   }
 
   @Test
@@ -888,9 +873,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |)
       """.stripMargin
 
-    tableEnv.sqlUpdate(ddl1)
+    tableEnv.executeSql(ddl1)
     assert(tableEnv.listTables().sameElements(Array[String]("t1")))
-    tableEnv.sqlUpdate("DROP TABLE IF EXISTS catalog1.database1.t1")
+    tableEnv.executeSql("DROP TABLE IF EXISTS catalog1.database1.t1")
     assert(tableEnv.listTables().sameElements(Array[String]("t1")))
   }
 
@@ -907,10 +892,10 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |  'k1' = 'v1'
         |)
       """.stripMargin
-    tableEnv.sqlUpdate(ddl1)
-    tableEnv.sqlUpdate("alter table t1 rename to t2")
+    tableEnv.executeSql(ddl1)
+    tableEnv.executeSql("alter table t1 rename to t2")
     assert(tableEnv.listTables().sameElements(Array[String]("t2")))
-    tableEnv.sqlUpdate("alter table t2 set ('k1' = 'a', 'k2' = 'b')")
+    tableEnv.executeSql("alter table t2 set ('k1' = 'a', 'k2' = 'b')")
     val expectedProperties = new util.HashMap[String, String]()
     expectedProperties.put("connector", "COLLECTION")
     expectedProperties.put("k1", "a")
@@ -921,14 +906,14 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     assertEquals(expectedProperties, properties)
     val currentCatalog = tableEnv.getCurrentCatalog
     val currentDB = tableEnv.getCurrentDatabase
-    tableEnv.sqlUpdate("alter table t2 add constraint ct1 primary key(a) not enforced")
+    tableEnv.executeSql("alter table t2 add constraint ct1 primary key(a) not enforced")
     val tableSchema1 = tableEnv.getCatalog(currentCatalog).get()
       .getTable(ObjectPath.fromString(s"${currentDB}.t2"))
       .getSchema
     assert(tableSchema1.getPrimaryKey.isPresent)
     assertEquals("CONSTRAINT ct1 PRIMARY KEY (a)",
       tableSchema1.getPrimaryKey.get().asSummaryString())
-    tableEnv.sqlUpdate("alter table t2 drop constraint ct1")
+    tableEnv.executeSql("alter table t2 drop constraint ct1")
     val tableSchema2 = tableEnv.getCatalog(currentCatalog).get()
       .getTable(ObjectPath.fromString(s"${currentDB}.t2"))
       .getSchema
@@ -939,9 +924,9 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
   def testUseCatalog(): Unit = {
     tableEnv.registerCatalog("cat1", new GenericInMemoryCatalog("cat1"))
     tableEnv.registerCatalog("cat2", new GenericInMemoryCatalog("cat2"))
-    tableEnv.sqlUpdate("use catalog cat1")
+    tableEnv.executeSql("use catalog cat1")
     assertEquals("cat1", tableEnv.getCurrentCatalog)
-    tableEnv.sqlUpdate("use catalog cat2")
+    tableEnv.executeSql("use catalog cat2")
     assertEquals("cat2", tableEnv.getCurrentCatalog)
   }
 
@@ -953,26 +938,26 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     val catalogDB2 = new CatalogDatabaseImpl(new util.HashMap[String, String](), "db2")
     catalog.createDatabase("db1", catalogDB1, true)
     catalog.createDatabase("db2", catalogDB2, true)
-    tableEnv.sqlUpdate("use cat1.db1")
+    tableEnv.executeSql("use cat1.db1")
     assertEquals("db1", tableEnv.getCurrentDatabase)
-    tableEnv.sqlUpdate("use db2")
+    tableEnv.executeSql("use db2")
     assertEquals("db2", tableEnv.getCurrentDatabase)
   }
 
   @Test
-  def testCreateDatabase: Unit = {
+  def testCreateDatabase(): Unit = {
     tableEnv.registerCatalog("cat1", new GenericInMemoryCatalog("default"))
     tableEnv.registerCatalog("cat2", new GenericInMemoryCatalog("default"))
-    tableEnv.sqlUpdate("use catalog cat1")
-    tableEnv.sqlUpdate("create database db1 ")
-    tableEnv.sqlUpdate("create database if not exists db1 ")
+    tableEnv.executeSql("use catalog cat1")
+    tableEnv.executeSql("create database db1 ")
+    tableEnv.executeSql("create database if not exists db1 ")
     try {
-      tableEnv.sqlUpdate("create database db1 ")
+      tableEnv.executeSql("create database db1 ")
       fail("ValidationException expected")
     } catch {
       case _: ValidationException => //ignore
     }
-    tableEnv.sqlUpdate("create database cat2.db1 comment 'test_comment'" +
+    tableEnv.executeSql("create database cat2.db1 comment 'test_comment'" +
                          " with ('k1' = 'v1', 'k2' = 'v2')")
     val database = tableEnv.getCatalog("cat2").get().getDatabase("db1")
     assertEquals("test_comment", database.getComment)
@@ -984,20 +969,20 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
   }
 
   @Test
-  def testDropDatabase: Unit = {
+  def testDropDatabase(): Unit = {
     tableEnv.registerCatalog("cat1", new GenericInMemoryCatalog("default"))
-    tableEnv.sqlUpdate("use catalog cat1")
-    tableEnv.sqlUpdate("create database db1")
-    tableEnv.sqlUpdate("drop database db1")
-    tableEnv.sqlUpdate("drop database if exists db1")
+    tableEnv.executeSql("use catalog cat1")
+    tableEnv.executeSql("create database db1")
+    tableEnv.executeSql("drop database db1")
+    tableEnv.executeSql("drop database if exists db1")
     try {
-      tableEnv.sqlUpdate("drop database db1")
+      tableEnv.executeSql("drop database db1")
       fail("ValidationException expected")
     } catch {
       case _: ValidationException => //ignore
     }
-    tableEnv.sqlUpdate("create database db1")
-    tableEnv.sqlUpdate("use db1")
+    tableEnv.executeSql("create database db1")
+    tableEnv.executeSql("use db1")
     val ddl1 =
       """
         |create table t1(
@@ -1008,7 +993,7 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |  'connector' = 'COLLECTION'
         |)
       """.stripMargin
-    tableEnv.sqlUpdate(ddl1)
+    tableEnv.executeSql(ddl1)
     val ddl2 =
       """
         |create table t2(
@@ -1019,22 +1004,22 @@ class CatalogTableITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |  'connector' = 'COLLECTION'
         |)
       """.stripMargin
-    tableEnv.sqlUpdate(ddl2)
+    tableEnv.executeSql(ddl2)
     try {
-      tableEnv.sqlUpdate("drop database db1")
+      tableEnv.executeSql("drop database db1")
       fail("ValidationException expected")
     } catch {
       case _: ValidationException => //ignore
     }
-    tableEnv.sqlUpdate("drop database db1 cascade")
+    tableEnv.executeSql("drop database db1 cascade")
   }
 
   @Test
-  def testAlterDatabase: Unit = {
+  def testAlterDatabase(): Unit = {
     tableEnv.registerCatalog("cat1", new GenericInMemoryCatalog("default"))
-    tableEnv.sqlUpdate("use catalog cat1")
-    tableEnv.sqlUpdate("create database db1 comment 'db1_comment' with ('k1' = 'v1')")
-    tableEnv.sqlUpdate("alter database db1 set ('k1' = 'a', 'k2' = 'b')")
+    tableEnv.executeSql("use catalog cat1")
+    tableEnv.executeSql("create database db1 comment 'db1_comment' with ('k1' = 'v1')")
+    tableEnv.executeSql("alter database db1 set ('k1' = 'a', 'k2' = 'b')")
     val database = tableEnv.getCatalog("cat1").get().getDatabase("db1")
     assertEquals("db1_comment", database.getComment)
     assertEquals(2, database.getProperties.size())
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableTest.scala
index 23ccb33..29a71c7 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogTableTest.scala
@@ -33,7 +33,7 @@ class CatalogTableTest {
 
   @Test
   def testDDLSchema(): Unit = {
-    tEnv.sqlUpdate(
+    tEnv.executeSql(
       """
         |CREATE TABLE t1 (
         |  f1 INT,
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala
index a68c4d8..3549b02 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/catalog/CatalogViewITCase.scala
@@ -19,13 +19,14 @@
 package org.apache.flink.table.planner.catalog
 
 import java.util
-
 import org.apache.flink.table.api.config.{ExecutionConfigOptions, TableConfigOptions}
 import org.apache.flink.table.api.{EnvironmentSettings, TableEnvironment}
 import org.apache.flink.table.api.internal.TableEnvironmentImpl
 import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil
 import org.apache.flink.test.util.AbstractTestBase
 import org.apache.flink.types.Row
+
 import org.junit.Assert.assertEquals
 import org.junit.{Before, Rule, Test}
 import org.junit.rules.ExpectedException
@@ -80,10 +81,6 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     row
   }
 
-  def execJob(name: String) = {
-    tableEnv.execute(name)
-  }
-
   @Test
   def testCreateViewIfNotExistsTwice(): Unit = {
     val sourceData = List(
@@ -135,8 +132,7 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     tableEnv.executeSql(viewWith2ColumnDDL)
 
     val result = tableEnv.sqlQuery(query)
-    result.insertInto("T2")
-    execJob("testJob")
+    TableEnvUtil.execInsertTableAndWaitResult(result, "T2")
     assertEquals(sourceData.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -185,8 +181,7 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     tableEnv.executeSql(viewDDL)
 
     val result = tableEnv.sqlQuery(query)
-    result.insertInto("T2")
-    execJob("testJob")
+    TableEnvUtil.execInsertTableAndWaitResult(result, "T2")
     assertEquals(sourceData.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -235,8 +230,7 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends AbstractTestBase {
     tableEnv.executeSql(viewDDL)
 
     val result = tableEnv.sqlQuery(query)
-    result.insertInto("T2")
-    execJob("testJob")
+    TableEnvUtil.execInsertTableAndWaitResult(result, "T2")
     assertEquals(sourceData.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
 
@@ -304,8 +298,7 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends AbstractTestBase {
 
     val query = "SELECT * FROM T3"
 
-    tableEnv.sqlQuery(query).insertInto("T2")
-    execJob("testJob")
+    TableEnvUtil.execInsertTableAndWaitResult(tableEnv.sqlQuery(query), "T2")
     // temporary view T3 masks permanent view T3
     assertEquals(temporaryViewData.sorted, TestCollectionTableFactory.RESULT.sorted)
 
@@ -317,8 +310,7 @@ class CatalogViewITCase(isStreamingMode: Boolean) extends AbstractTestBase {
         |DROP TEMPORARY VIEW IF EXISTS T3
       """.stripMargin
     tableEnv.executeSql(dropTemporaryView)
-    tableEnv.sqlQuery(query).insertInto("T2")
-    execJob("testJob")
+    TableEnvUtil.execInsertTableAndWaitResult(tableEnv.sqlQuery(query), "T2")
     // now we only have permanent view T3
     assertEquals(permanentViewData.sorted, TestCollectionTableFactory.RESULT.sorted)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
index 6aa2ff4..6e1da9c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/expressions/utils/ExpressionTestBase.scala
@@ -239,7 +239,7 @@ abstract class ExpressionTestBase {
   private def addTableApiTestExpr(tableApiExpr: Expression, expected: String): Unit = {
     // create RelNode from Table API expression
     val relNode = relBuilder
-        .queryOperation(tEnv.scan(tableName).select(tableApiExpr).getQueryOperation).build()
+        .queryOperation(tEnv.from(tableName).select(tableApiExpr).getQueryOperation).build()
 
     addTestExpr(relNode, expected, tableApiExpr.asSummaryString())
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala
index a40f3ba..331f52a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/DagOptimizationTest.scala
@@ -43,8 +43,7 @@ class DagOptimizationTest extends TableTestBase {
   def testSingleSink1(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT c, COUNT(a) AS cnt FROM MyTable GROUP BY c")
     val appendSink = util.createCollectTableSink(Array("c", "cnt"), Array(STRING, LONG))
-    util.writeToSink(table, appendSink, "appendSink")
-    util.verifyPlan()
+    util.verifyPlanInsert(table, appendSink, "appendSink")
   }
 
   @Test
@@ -62,8 +61,7 @@ class DagOptimizationTest extends TableTestBase {
     val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
 
     val appendSink = util.createCollectTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
-    util.writeToSink(table6, appendSink, "appendSink")
-    util.verifyPlan()
+    util.verifyPlanInsert(table6, appendSink, "appendSink")
   }
 
   @Test
@@ -76,8 +74,7 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
 
     val appendSink = util.createCollectTableSink(Array("a1", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, appendSink, "appendSink")
-    util.verifyPlan()
+    util.verifyPlanInsert(table3, appendSink, "appendSink")
   }
 
   @Test
@@ -97,8 +94,7 @@ class DagOptimizationTest extends TableTestBase {
     val table7 = util.tableEnv.sqlQuery("SELECT a1, b1, c1 FROM table1, table6 WHERE a1 = a3")
 
     val sink = util.createCollectTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table7, sink, "sink")
-    util.verifyPlan()
+    util.verifyPlanInsert(table7, sink, "sink")
   }
 
   @Test
@@ -117,9 +113,7 @@ class DagOptimizationTest extends TableTestBase {
     val sink = util.createCollectTableSink(
       Array("a", "b", "c", "d", "e", "f", "i", "j", "k", "l", "m", "s"),
       Array(INT, LONG, STRING, INT, LONG, STRING, INT, LONG, INT, STRING, LONG, STRING))
-    util.writeToSink(table, sink, "sink")
-
-    util.verifyPlan()
+    util.verifyPlanInsert(table, sink, "sink")
   }
 
   @Test
@@ -128,13 +122,12 @@ class DagOptimizationTest extends TableTestBase {
       "(SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
     val table = util.tableEnv.sqlQuery(sqlQuery)
     val sink = util.createCollectTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table, sink, "sink")
-
-    util.verifyPlan()
+    util.verifyPlanInsert(table, sink, "sink")
   }
 
   @Test
   def testMultiSinks1(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_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")
@@ -143,16 +136,19 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(sum_a) AS total_min FROM table1")
 
     val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table2)
 
     val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinks2(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
@@ -164,16 +160,19 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
 
     val sink1 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table3)
 
     val sink2 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinks3(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
@@ -185,16 +184,19 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
 
     val sink1 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table2, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table2)
 
     val sink2 = util.createCollectTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinks4(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
 
@@ -211,16 +213,19 @@ class DagOptimizationTest extends TableTestBase {
     val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
 
     val sink1 = util.createCollectTableSink(Array("a1", "b", "c2"), Array(INT, LONG, STRING))
-    util.writeToSink(table5, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table5)
 
     val sink2 = util.createCollectTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
-    util.writeToSink(table6, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table6)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinks5(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     // test with non-deterministic udf
@@ -231,23 +236,27 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM table1")
 
     val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table2)
 
     val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiLevelViews(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
     util.tableEnv.registerTable("TempTable1", table1)
     val sink1 = util.createCollectTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table1, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%world%'")
     util.tableEnv.registerTable("TempTable2", table2)
@@ -266,17 +275,20 @@ class DagOptimizationTest extends TableTestBase {
 
     val table4 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b < 4")
     val sink2 = util.createCollectTableSink(Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table4, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table4)
 
     val table5 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b >=4 AND b < 6")
     val sink3 = util.createCollectTableSink(Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table5, sink3, "sink3")
+    util.tableEnv.registerTableSink("sink3", sink3)
+    stmtSet.addInsert("sink3", table5)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinksWithUDTF(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addFunction("split", new TableFunc1)
@@ -304,18 +316,21 @@ class DagOptimizationTest extends TableTestBase {
     val sqlQuery5 = "SELECT * FROM table4 WHERE a > 50"
     val table5 = util.tableEnv.sqlQuery(sqlQuery5)
     val sink1 = util.createCollectTableSink(Array("a", "total_c"), Array(INT, LONG))
-    util.writeToSink(table5, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table5)
 
     val sqlQuery6 = "SELECT * FROM table4 WHERE a < 50"
     val table6 = util.tableEnv.sqlQuery(sqlQuery6)
     val sink2 = util.createCollectTableSink(Array("a", "total_c"), Array(INT, LONG))
-    util.writeToSink(table6, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table6)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinksSplitOnUnion1(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
@@ -325,17 +340,20 @@ class DagOptimizationTest extends TableTestBase {
 
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
     val sink1 = util.createCollectTableSink( Array("total_sum"), Array(INT))
-    util.writeToSink(table1, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table1)
 
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
     val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinksSplitOnUnion2(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     util.tableEnv.getConfig.getConfiguration.setBoolean(
@@ -355,22 +373,26 @@ class DagOptimizationTest extends TableTestBase {
 
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
     val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table1, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
     val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table2, sink2, "sink2")
+     util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table2)
 
     val sqlQuery2 = "SELECT a FROM (SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
     val table3 = util.tableEnv.sqlQuery(sqlQuery2)
     val sink3 = util.createCollectTableSink(Array("a"), Array(INT))
-    util.writeToSink(table3, sink3, "sink3")
+    util.tableEnv.registerTableSink("sink3", sink3)
+    stmtSet.addInsert("sink3", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinksSplitOnUnion3(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
@@ -380,7 +402,8 @@ class DagOptimizationTest extends TableTestBase {
     util.tableEnv.registerTable("TempTable", table)
 
     val sink1 = util.createCollectTableSink(Array("a", "c"), Array(INT, STRING))
-    util.writeToSink(table, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table)
 
     val sqlQuery2 = "SELECT a, c FROM TempTable UNION ALL SELECT a, c FROM MyTable2"
     val table1 = util.tableEnv.sqlQuery(sqlQuery2)
@@ -388,17 +411,20 @@ class DagOptimizationTest extends TableTestBase {
 
     val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable1")
     val sink2 = util.createCollectTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, sink2, "sink2")
+     util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table2)
 
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable1")
     val sink3 = util.createCollectTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, sink3, "sink3")
+    util.tableEnv.registerTableSink("sink3", sink3)
+    stmtSet.addInsert("sink3", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinksSplitOnUnion4(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
@@ -416,17 +442,20 @@ class DagOptimizationTest extends TableTestBase {
 
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
     val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table1, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
     val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table2, sink2, "sink2")
+     util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table2)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testMultiSinksWithWindow(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.addTableSource[(Int, Double, Int, Timestamp)]("MyTable2", 'a, 'b, 'c, 'ts)
     val sqlQuery1 =
       """
@@ -458,14 +487,16 @@ class DagOptimizationTest extends TableTestBase {
     val sink1 = util.createCollectTableSink(
       Array("a", "sum_c", "time", "window_start", "window_end"),
       Array(INT, DOUBLE, TIMESTAMP, TIMESTAMP, TIMESTAMP))
-    util.writeToSink(table1, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table1)
 
     val table2 = util.tableEnv.sqlQuery(sqlQuery2)
     val sink2 = util.createCollectTableSink(Array("a", "sum_c", "time"),
       Array(INT, DOUBLE, TIMESTAMP))
-    util.writeToSink(table2, sink2, "sink2")
+     util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table2)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala
index 94bfeee..ed1c336 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/PartitionableSinkTest.scala
@@ -32,7 +32,7 @@ class PartitionableSinkTest extends TableTestBase {
   createTable("sink", shuffleBy = false)
 
   private def createTable(name: String, shuffleBy: Boolean): Unit = {
-    util.tableEnv.sqlUpdate(
+    util.tableEnv.executeSql(
       s"""
          |create table $name (
          |  a bigint,
@@ -49,33 +49,33 @@ class PartitionableSinkTest extends TableTestBase {
 
   @Test
   def testStatic(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1, c=1) SELECT a FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (b=1, c=1) SELECT a FROM MyTable")
   }
 
   @Test
   def testDynamic(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink SELECT a, b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink SELECT a, b, c FROM MyTable")
   }
 
   @Test
   def testDynamicShuffleBy(): Unit = {
     createTable("sinkShuffleBy", shuffleBy = true)
-    util.verifySqlUpdate("INSERT INTO sinkShuffleBy SELECT a, b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sinkShuffleBy SELECT a, b, c FROM MyTable")
   }
 
   @Test
   def testPartial(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable")
   }
 
   @Test(expected = classOf[ValidationException])
   def testWrongStatic(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable")
   }
 
   @Test(expected = classOf[ValidationException])
   def testWrongFields(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable")
   }
 
   @Test
@@ -84,6 +84,6 @@ class PartitionableSinkTest extends TableTestBase {
     thrown.expectMessage(
       "INSERT INTO <table> PARTITION statement only support SELECT clause for now," +
           " 'VALUES ROW(5)' is not supported yet")
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1, c=1) VALUES (5)")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (b=1, c=1) VALUES (5)")
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala
index 4a91621..bee2eba 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/SinkTest.scala
@@ -38,12 +38,12 @@ class SinkTest extends TableTestBase {
   def testSingleSink(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT COUNT(*) AS cnt FROM MyTable GROUP BY a")
     val sink = util.createCollectTableSink(Array("a"), Array(LONG))
-    util.writeToSink(table, sink, "sink")
-    util.verifyPlan()
+    util.verifyPlanInsert(table, sink, "sink")
   }
 
   @Test
   def testMultiSinks(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_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")
@@ -52,11 +52,13 @@ class SinkTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(sum_a) AS total_min FROM table1")
 
     val sink1 = util.createCollectTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, sink1, "sink1")
+    util.tableEnv.registerTableSink("sink1", sink1)
+    stmtSet.addInsert("sink1", table2)
 
     val sink2 = util.createCollectTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, sink2, "sink2")
+    util.tableEnv.registerTableSink("sink2", sink2)
+    stmtSet.addInsert("sink2", table3)
 
-    util.verifyPlan()
+    util.verifyPlan(stmtSet)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala
index fba8d95..b27164b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/batch/sql/agg/WindowAggregateTest.scala
@@ -46,7 +46,7 @@ class WindowAggregateTest(aggStrategy: AggregatePhaseStrategy) extends TableTest
     util.addTableSource[(Int, Timestamp, Int, Long)]("MyTable", 'a, 'b, 'c, 'd)
     util.addTableSource[(Timestamp, Long, Int, String)]("MyTable1", 'ts, 'a, 'b, 'c)
     util.addTableSource[(Int, Long, String, Int, Timestamp)]("MyTable2", 'a, 'b, 'c, 'd, 'ts)
-    util.tableEnv.sqlUpdate(
+    util.tableEnv.executeSql(
       s"""
          |create table MyTable3 (
          |  a int,
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala
index 1a227f1..4a66708 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/common/TableFactoryTest.scala
@@ -22,6 +22,7 @@ import org.apache.flink.table.catalog.{GenericInMemoryCatalog, ObjectIdentifier}
 import org.apache.flink.table.factories.TableFactory
 import org.apache.flink.table.planner.factories.utils.TestCollectionTableFactory
 import org.apache.flink.table.planner.plan.utils.TestContextTableFactory
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil
 import org.apache.flink.table.planner.utils.TableTestBase
 
 import org.junit.runner.RunWith
@@ -82,11 +83,10 @@ class TableFactoryTest(isBatch: Boolean) extends TableTestBase {
         |insert into t2
         |select t1.a, t1.c from t1
       """.stripMargin
-    util.tableEnv.sqlUpdate(sourceDDL)
-    util.tableEnv.sqlUpdate(sinkDDL)
-    util.tableEnv.sqlUpdate(query)
+    util.tableEnv.executeSql(sourceDDL)
+    util.tableEnv.executeSql(sinkDDL)
 
-    util.tableEnv.explain(false)
+    util.tableEnv.explainSql(query)
     Assert.assertTrue(factory.hasInvokedSource)
     Assert.assertTrue(factory.hasInvokedSink)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala
index d7ac402..1b7a2c4 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/hint/OptionsHintTest.scala
@@ -24,7 +24,7 @@ import org.apache.flink.table.catalog.{CatalogViewImpl, ObjectPath}
 import org.apache.flink.table.planner.JHashMap
 import org.apache.flink.table.planner.plan.hint.OptionsHintTest.{IS_BOUNDED, Param}
 import org.apache.flink.table.planner.plan.nodes.calcite.LogicalSink
-import org.apache.flink.table.planner.utils.{OptionsTableSink, TableTestBase, TableTestUtil, TestingTableEnvironment}
+import org.apache.flink.table.planner.utils.{OptionsTableSink, TableTestBase, TableTestUtil, TestingStatementSet, TestingTableEnvironment}
 
 import org.hamcrest.Matchers._
 import org.junit.Assert.{assertEquals, assertThat}
@@ -93,9 +93,10 @@ class OptionsHintTest(param: Param)
          |insert into t1 /*+ OPTIONS(k1='#v1', k5='v5') */
          |select d, e from t2
          |""".stripMargin
-    util.tableEnv.sqlUpdate(sql)
-    val testTableEnv = util.tableEnv.asInstanceOf[TestingTableEnvironment]
-    val relNodes = testTableEnv.getBufferedOperations.map(util.getPlanner.translateToRel)
+    val stmtSet = util.tableEnv.createStatementSet()
+    stmtSet.addInsertSql(sql)
+    val testStmtSet = stmtSet.asInstanceOf[TestingStatementSet]
+    val relNodes = testStmtSet.getOperations.map(util.getPlanner.translateToRel)
     assertThat(relNodes.length, is(1))
     assert(relNodes.head.isInstanceOf[LogicalSink])
     val sink = relNodes.head.asInstanceOf[LogicalSink]
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala
index 6151505..cd36b2d 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushFilterIntoLegacyTableSourceScanRuleTest.scala
@@ -66,7 +66,7 @@ class PushFilterIntoLegacyTableSourceScanRuleTest extends TableTestBase {
          |  'is-bounded' = 'true'
          |)
        """.stripMargin
-    util.tableEnv.sqlUpdate(ddl)
+    util.tableEnv.executeSql(ddl)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala
index a38ff10..c62c8d8 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoLegacyTableSourceScanRuleTest.scala
@@ -60,7 +60,7 @@ class PushProjectIntoLegacyTableSourceScanRuleTest extends TableTestBase {
          |  'is-bounded' = 'true'
          |)
        """.stripMargin
-    util.tableEnv.sqlUpdate(ddl1)
+    util.tableEnv.executeSql(ddl1)
 
     val ddl2 =
       s"""
@@ -74,7 +74,7 @@ class PushProjectIntoLegacyTableSourceScanRuleTest extends TableTestBase {
          |  'is-bounded' = 'true'
          |)
        """.stripMargin
-    util.tableEnv.sqlUpdate(ddl2)
+    util.tableEnv.executeSql(ddl2)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala
index b405b57..07aac62 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/rules/physical/stream/ChangelogModeInferenceTest.scala
@@ -20,13 +20,15 @@ package org.apache.flink.table.planner.plan.rules.physical.stream
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.runtime.utils.StreamingWithAggTestBase.{AggMode, LocalGlobalOff, LocalGlobalOn}
 import org.apache.flink.table.planner.utils.{AggregatePhaseStrategy, TableTestBase}
-import org.junit.{Before, Test}
+
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
+import org.junit.{Before, Test}
 
 import java.util
 
@@ -51,13 +53,13 @@ class ChangelogModeInferenceTest(aggMode: AggMode) extends TableTestBase {
 
   @Test
   def testSelect(): Unit = {
-    util.verifyPlanWithTrait("SELECT word, number FROM MyTable")
+    util.verifyPlan("SELECT word, number FROM MyTable", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testOneLevelGroupBy(): Unit = {
     // one level unbounded groupBy
-    util.verifyPlanWithTrait("SELECT COUNT(number) FROM MyTable GROUP BY word")
+    util.verifyPlan("SELECT COUNT(number) FROM MyTable GROUP BY word", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -69,7 +71,7 @@ class ChangelogModeInferenceTest(aggMode: AggMode) extends TableTestBase {
         |  SELECT word, COUNT(number) as cnt FROM MyTable GROUP BY word
         |) GROUP BY cnt
       """.stripMargin
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -84,7 +86,7 @@ class ChangelogModeInferenceTest(aggMode: AggMode) extends TableTestBase {
         |   SELECT word, cnt FROM MyTable2
         |) GROUP BY cnt
       """.stripMargin
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
index 3c0e322..0a05700 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DagOptimizationTest.scala
@@ -18,6 +18,7 @@
 package org.apache.flink.table.planner.plan.stream.sql
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.plan.optimize.RelNodeBlockPlanBuilder
 import org.apache.flink.table.planner.runtime.utils.JavaUserDefinedScalarFunctions.NonDeterministicUdf
@@ -39,8 +40,7 @@ class DagOptimizationTest extends TableTestBase {
   def testSingleSink1(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT c, COUNT(a) AS cnt FROM MyTable GROUP BY c")
     val retractSink = util.createRetractTableSink(Array("c", "cnt"), Array(STRING, LONG))
-    util.writeToSink(table, retractSink, "retractSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, retractSink, "retractSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -58,8 +58,7 @@ class DagOptimizationTest extends TableTestBase {
     val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
 
     val appendSink = util.createAppendTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
-    util.writeToSink(table6, appendSink, "appendSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table6, appendSink, "appendSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -72,8 +71,7 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
 
     val appendSink = util.createAppendTableSink(Array("a1", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, appendSink, "appendSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table3, appendSink, "appendSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -93,8 +91,7 @@ class DagOptimizationTest extends TableTestBase {
     val table7 = util.tableEnv.sqlQuery("SELECT a1, b1, c1 FROM table1, table6 WHERE a1 = a3")
 
     val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table7, appendSink, "appendSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table7, appendSink, "appendSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -113,9 +110,7 @@ class DagOptimizationTest extends TableTestBase {
     val appendSink = util.createAppendTableSink(
       Array("a", "b", "c", "d", "e", "f", "i", "j", "k", "l", "m", "s"),
       Array(INT, LONG, STRING, INT, LONG, STRING, INT, LONG, INT, STRING, LONG, STRING))
-    util.writeToSink(table, appendSink, "appendSink")
-
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, appendSink, "appendSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -127,13 +122,12 @@ class DagOptimizationTest extends TableTestBase {
       "(SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
     val table = util.tableEnv.sqlQuery(sqlQuery)
     val retractSink = util.createRetractTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table, retractSink, "retractSink")
-
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, retractSink, "retractSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinks1(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_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")
@@ -142,16 +136,19 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(sum_a) AS total_min FROM table1")
 
     val retractSink1 = util.createRetractTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, retractSink1, "retractSink1")
+    util.tableEnv.registerTableSink("retractSink1", retractSink1)
+    stmtSet.addInsert("retractSink1", table2)
 
     val retractSink2 = util.createRetractTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, retractSink2, "retractSink2")
+    util.tableEnv.registerTableSink("retractSink2", retractSink2)
+    stmtSet.addInsert("retractSink2", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinks2(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
@@ -163,16 +160,19 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
 
     val appendSink1 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, appendSink1, "appendSink1")
+    util.tableEnv.registerTableSink("appendSink1", appendSink1)
+    stmtSet.addInsert("appendSink1", table3)
 
     val appendSink2 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, appendSink2, "appendSink2")
+    util.tableEnv.registerTableSink("appendSink2", appendSink2)
+    stmtSet.addInsert("appendSink2", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinks3(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, false)
     util.addTableSource[(Int, Long, String, Double, Boolean)]("MyTable2", 'a, 'b, 'c, 'd, 'e)
@@ -184,16 +184,19 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT * FROM table1 UNION ALL SELECT * FROM table2")
 
     val appendSink1 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table2, appendSink1, "appendSink1")
+    util.tableEnv.registerTableSink("appendSink1", appendSink1)
+    stmtSet.addInsert("appendSink1", table2)
 
     val appendSink2 = util.createAppendTableSink(Array("a", "b1"), Array(INT, LONG))
-    util.writeToSink(table3, appendSink2, "appendSink2")
+    util.tableEnv.registerTableSink("appendSink2", appendSink2)
+    stmtSet.addInsert("appendSink2", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinks4(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val table1 = util.tableEnv.sqlQuery("SELECT a as a1, b FROM MyTable WHERE a <= 10")
     util.tableEnv.registerTable("table1", table1)
     val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE a >= 0")
@@ -207,16 +210,19 @@ class DagOptimizationTest extends TableTestBase {
     val table6 = util.tableEnv.sqlQuery("SELECT a1, b, c1 FROM table4, table5 WHERE a1 = a3")
 
     val appendSink1 = util.createAppendTableSink(Array("a1", "b", "c2"), Array(INT, LONG, STRING))
-    util.writeToSink(table5, appendSink1, "appendSink1")
+    util.tableEnv.registerTableSink("appendSink1", appendSink1)
+    stmtSet.addInsert("appendSink1", table5)
 
     val appendSink2 = util.createAppendTableSink(Array("a1", "b", "c1"), Array(INT, LONG, STRING))
-    util.writeToSink(table6, appendSink2, "appendSink2")
+    util.tableEnv.registerTableSink("appendSink2", appendSink2)
+    stmtSet.addInsert("appendSink2", table6)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinks5(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     // test with non-deterministic udf
@@ -227,16 +233,19 @@ class DagOptimizationTest extends TableTestBase {
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM table1")
 
     val retractSink1 = util.createRetractTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, retractSink1, "retractSink1")
+    util.tableEnv.registerTableSink("retractSink1", retractSink1)
+    stmtSet.addInsert("retractSink1", table2)
 
     val retractSink2 = util.createRetractTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, retractSink2, "retractSink2")
+    util.tableEnv.registerTableSink("retractSink2", retractSink2)
+    stmtSet.addInsert("retractSink2", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinksWithUDTF(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addFunction("split", new TableFunc1)
@@ -264,18 +273,21 @@ class DagOptimizationTest extends TableTestBase {
     val sqlQuery5 = "SELECT * FROM table4 WHERE a > 50"
     val table5 = util.tableEnv.sqlQuery(sqlQuery5)
     val retractSink1 = util.createRetractTableSink(Array("a", "total_c"), Array(INT, LONG))
-    util.writeToSink(table5, retractSink1, "retractSink1")
+    util.tableEnv.registerTableSink("retractSink1", retractSink1)
+    stmtSet.addInsert("retractSink1", table5)
 
     val sqlQuery6 = "SELECT * FROM table4 WHERE a < 50"
     val table6 = util.tableEnv.sqlQuery(sqlQuery6)
     val retractSink2 = util.createRetractTableSink(Array("a", "total_c"), Array(INT, LONG))
-    util.writeToSink(table6, retractSink2, "retractSink2")
+    util.tableEnv.registerTableSink("retractSink2", retractSink2)
+    stmtSet.addInsert("retractSink2", table6)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinksSplitOnUnion1(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
@@ -285,17 +297,20 @@ class DagOptimizationTest extends TableTestBase {
 
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
     val upsertSink = util.createUpsertTableSink(Array(), Array("total_sum"), Array(INT))
-    util.writeToSink(table1, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table1)
 
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
     val retractSink = util.createRetractTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table3, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinksSplitOnUnion2(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_REUSE_OPTIMIZE_BLOCK_WITH_DIGEST_ENABLED, true)
     util.tableEnv.getConfig.getConfiguration.setBoolean(
@@ -315,22 +330,26 @@ class DagOptimizationTest extends TableTestBase {
 
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
     val retractSink1 = util.createRetractTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table1, retractSink1, "retractSink1")
+    util.tableEnv.registerTableSink("retractSink1", retractSink1)
+    stmtSet.addInsert("retractSink1", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
     val retractSink2 = util.createRetractTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table2, retractSink2, "retractSink2")
+    util.tableEnv.registerTableSink("retractSink2", retractSink2)
+    stmtSet.addInsert("retractSink2", table2)
 
     val sqlQuery2 = "SELECT a FROM (SELECT a, c FROM MyTable UNION ALL SELECT d, f FROM MyTable1)"
     val table3 = util.tableEnv.sqlQuery(sqlQuery2)
     val appendSink3 = util.createAppendTableSink(Array("a"), Array(INT))
-    util.writeToSink(table3, appendSink3, "appendSink3")
+    util.tableEnv.registerTableSink("appendSink3", appendSink3)
+    stmtSet.addInsert("appendSink3", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinksSplitOnUnion3(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
@@ -340,7 +359,8 @@ class DagOptimizationTest extends TableTestBase {
     util.tableEnv.registerTable("TempTable", table)
 
     val appendSink = util.createAppendTableSink(Array("a", "c"), Array(INT, STRING))
-    util.writeToSink(table, appendSink, "appendSink")
+    util.tableEnv.registerTableSink("appendSink", appendSink)
+    stmtSet.addInsert("appendSink", table)
 
     val sqlQuery2 = "SELECT a, c FROM TempTable UNION ALL SELECT a, c FROM MyTable2"
     val table1 = util.tableEnv.sqlQuery(sqlQuery2)
@@ -348,17 +368,20 @@ class DagOptimizationTest extends TableTestBase {
 
     val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable1")
     val retractSink = util.createRetractTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table2)
 
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable1")
     val upsertSink = util.createUpsertTableSink(Array(), Array("total_min"), Array(INT))
-    util.writeToSink(table3, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiSinksSplitOnUnion4(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
     util.addTableSource[(Int, Long, String)]("MyTable2", 'a, 'b, 'c)
@@ -376,13 +399,15 @@ class DagOptimizationTest extends TableTestBase {
 
     val table1 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable")
     val upsertSink = util.createUpsertTableSink(Array(), Array("total_sum"), Array(INT))
-    util.writeToSink(table1, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable")
     val retractSink = util.createRetractTableSink(Array("total_min"), Array(INT))
-    util.writeToSink(table2, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table2)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -397,13 +422,12 @@ class DagOptimizationTest extends TableTestBase {
 
     val upsertSink = util.createUpsertTableSink(Array(), Array("b", "c", "a_sum"),
       Array(LONG, STRING, INT))
-    util.writeToSink(table, upsertSink, "upsertSink")
-
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, upsertSink, "upsertSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testUpdateAsRetractConsumedAtSinkBlock(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val table = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable")
     util.tableEnv.registerTable("TempTable", table)
 
@@ -418,17 +442,20 @@ class DagOptimizationTest extends TableTestBase {
     val table1 = util.tableEnv.sqlQuery(sqlQuery)
     val retractSink = util.createRetractTableSink(
       Array("a", "b", "c", "rank_num"), Array(INT, LONG, STRING, LONG))
-    util.writeToSink(table1, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table1)
 
     val upsertSink = util.createUpsertTableSink(Array(), Array("a", "b"), Array(INT, LONG))
     val table2 = util.tableEnv.sqlQuery("SELECT a, b FROM TempTable WHERE a < 6")
-    util.writeToSink(table2, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table2)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testUpdateAsRetractConsumedAtSourceBlock(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val sqlQuery =
       s"""
          |SELECT * FROM (
@@ -442,24 +469,28 @@ class DagOptimizationTest extends TableTestBase {
 
     val table1 = util.tableEnv.sqlQuery("SELECT a FROM TempTable WHERE a > 6")
     val retractSink = util.createRetractTableSink(Array("a"), Array(INT))
-    util.writeToSink(table1, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT a, b FROM TempTable WHERE a < 6")
     val upsertSink = util.createUpsertTableSink(Array(), Array("a", "b"), Array(INT, LONG))
-    util.writeToSink(table2, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table2)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMultiLevelViews(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
     util.tableEnv.registerTable("TempTable1", table1)
     val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table1, appendSink, "appendSink")
+    util.tableEnv.registerTableSink("appendSink", appendSink)
+    stmtSet.addInsert("appendSink", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%world%'")
     util.tableEnv.registerTable("TempTable2", table2)
@@ -478,24 +509,28 @@ class DagOptimizationTest extends TableTestBase {
 
     val table4 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b < 4")
     val retractSink = util.createRetractTableSink(Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table4, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table4)
 
     val table5 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable3 WHERE b >=4 AND b < 6")
     val upsertSink = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table5, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table5)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testSharedUnionNode(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.tableEnv.getConfig.getConfiguration.setBoolean(
       RelNodeBlockPlanBuilder.TABLE_OPTIMIZER_UNIONALL_AS_BREAKPOINT_DISABLED, true)
 
     val table1 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%hello%'")
     util.tableEnv.registerTable("TempTable1", table1)
     val appendSink = util.createAppendTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table1, appendSink, "appendSink")
+    util.tableEnv.registerTableSink("appendSink", appendSink)
+    stmtSet.addInsert("appendSink", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT a, b, c FROM MyTable WHERE c LIKE '%world%'")
     util.tableEnv.registerTable("TempTable2", table2)
@@ -511,20 +546,23 @@ class DagOptimizationTest extends TableTestBase {
 
     val table4 = util.tableEnv.sqlQuery("SELECT * FROM TempTable3 WHERE b >= 5")
     val retractSink1 = util.createRetractTableSink(Array("a", "b", "c"), Array(INT, LONG, STRING))
-    util.writeToSink(table4, retractSink1, "retractSink1")
+    util.tableEnv.registerTableSink("retractSink1", retractSink1)
+    stmtSet.addInsert("retractSink1", table4)
 
     val table5 = util.tableEnv.sqlQuery("SELECT b, count(a) as cnt FROM TempTable3 GROUP BY b")
     util.tableEnv.registerTable("TempTable4", table5)
 
     val table6 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable4 WHERE b < 4")
     val retractSink2 = util.createRetractTableSink(Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table6, retractSink2, "retractSink2")
+    util.tableEnv.registerTableSink("retractSink2", retractSink2)
+    stmtSet.addInsert("retractSink2", table6)
 
     util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable4 WHERE b >=4 AND b < 6")
     val upsertSink = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table6, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table6)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
index 6671ae5..973b326 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/MiniBatchIntervalInferTest.scala
@@ -58,7 +58,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   def testRedundantWatermarkDefinition(): Unit = {
     util.tableEnv.getConfig.getConfiguration
         .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
-    util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("MyTable3", util.tableEnv.from("MyTable1"), "rowtime", 0)
     val sql = "SELECT b, COUNT(DISTINCT a), MAX(b), SUM(c) FROM MyTable3 GROUP BY b"
     util.verifyPlan(sql)
   }
@@ -69,7 +69,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
     tableConfig.getConfiguration
         .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
     withEarlyFireDelay(tableConfig, Time.milliseconds(500))
-    util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("MyTable3", util.tableEnv.from("MyTable1"), "rowtime", 0)
     val sql =
       """
         | SELECT b, SUM(cnt)
@@ -90,7 +90,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   def testWindowCascade(): Unit = {
     util.tableEnv.getConfig.getConfiguration
         .setString(ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "3 s")
-    util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("MyTable3", util.tableEnv.from("MyTable1"), "rowtime", 0)
     val sql =
       """
         | SELECT b,
@@ -109,8 +109,8 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testWindowJoinWithMiniBatch(): Unit = {
-    util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
-    util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
+    util.addTableWithWatermark("LeftT", util.tableEnv.from("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("RightT", util.tableEnv.from("MyTable2"), "rowtime", 0)
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
@@ -132,7 +132,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testRowtimeRowsOverWithMiniBatch(): Unit = {
-    util.addTableWithWatermark("MyTable3", util.tableEnv.scan("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("MyTable3", util.tableEnv.from("MyTable1"), "rowtime", 0)
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
@@ -152,15 +152,15 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testTemporalTableFunctionJoinWithMiniBatch(): Unit = {
-    util.addTableWithWatermark("Orders", util.tableEnv.scan("MyTable1"), "rowtime", 0)
-    util.addTableWithWatermark("RatesHistory", util.tableEnv.scan("MyTable2"), "rowtime", 0)
+    util.addTableWithWatermark("Orders", util.tableEnv.from("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("RatesHistory", util.tableEnv.from("MyTable2"), "rowtime", 0)
 
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
     util.addFunction(
       "Rates",
-      util.tableEnv.scan("RatesHistory").createTemporalTableFunction($"rowtime", $"b"))
+      util.tableEnv.from("RatesHistory").createTemporalTableFunction($"rowtime", $"b"))
 
     val sqlQuery =
       """
@@ -180,8 +180,8 @@ class MiniBatchIntervalInferTest extends TableTestBase {
   @Test
   def testMultiOperatorNeedsWatermark1(): Unit = {
     // infer result: miniBatchInterval=[Rowtime, 0ms]
-    util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
-    util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
+    util.addTableWithWatermark("LeftT", util.tableEnv.from("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("RightT", util.tableEnv.from("MyTable2"), "rowtime", 0)
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "1 s")
 
@@ -206,8 +206,8 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testMultiOperatorNeedsWatermark2(): Unit = {
-    util.addTableWithWatermark("LeftT", util.tableEnv.scan("MyTable1"), "rowtime", 0)
-    util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
+    util.addTableWithWatermark("LeftT", util.tableEnv.from("MyTable1"), "rowtime", 0)
+    util.addTableWithWatermark("RightT", util.tableEnv.from("MyTable2"), "rowtime", 0)
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
 
@@ -243,7 +243,7 @@ class MiniBatchIntervalInferTest extends TableTestBase {
 
   @Test
   def testMultiOperatorNeedsWatermark3(): Unit = {
-    util.addTableWithWatermark("RightT", util.tableEnv.scan("MyTable2"), "rowtime", 0)
+    util.addTableWithWatermark("RightT", util.tableEnv.from("MyTable2"), "rowtime", 0)
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "6 s")
 
@@ -273,12 +273,13 @@ class MiniBatchIntervalInferTest extends TableTestBase {
     */
   @Test
   def testMultipleWindowAggregates(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.addDataStream[(Int, Long, String)]("T1", 'id1, 'rowtime.rowtime, 'text)
     util.addDataStream[(Int, Long, Int, String, String)](
       "T2",
       'id2, 'rowtime.rowtime, 'cnt, 'name, 'goods)
-    util.addTableWithWatermark("T3", util.tableEnv.scan("T1"), "rowtime", 0)
-    util.addTableWithWatermark("T4", util.tableEnv.scan("T2"), "rowtime", 0)
+    util.addTableWithWatermark("T3", util.tableEnv.from("T1"), "rowtime", 0)
+    util.addTableWithWatermark("T4", util.tableEnv.from("T2"), "rowtime", 0)
 
     util.tableEnv.getConfig.getConfiguration.setString(
       ExecutionConfigOptions.TABLE_EXEC_MINIBATCH_ALLOW_LATENCY, "500 ms")
@@ -313,7 +314,8 @@ class MiniBatchIntervalInferTest extends TableTestBase {
         |GROUP BY HOP(ts, INTERVAL '12' SECOND, INTERVAL '4' SECOND), id1
       """.stripMargin)
     val appendSink1 = util.createAppendTableSink(Array("a", "b"), Array(INT, STRING))
-    util.writeToSink(table3, appendSink1, "appendSink1")
+    util.tableEnv.registerTableSink("appendSink1", appendSink1)
+    stmtSet.addInsert("appendSink1", table3)
 
     val table4 = util.tableEnv.sqlQuery(
       """
@@ -323,7 +325,8 @@ class MiniBatchIntervalInferTest extends TableTestBase {
         |GROUP BY TUMBLE(ts, INTERVAL '9' SECOND), id1
       """.stripMargin)
     val appendSink2 = util.createAppendTableSink(Array("a", "b"), Array(INT, STRING))
-    util.writeToSink(table4, appendSink2, "appendSink2")
+    util.tableEnv.registerTableSink("appendSink2", appendSink2)
+    stmtSet.addInsert("appendSink2", table4)
 
     val table5 = util.tableEnv.sqlQuery(
       """
@@ -333,9 +336,10 @@ class MiniBatchIntervalInferTest extends TableTestBase {
         |GROUP BY id1
       """.stripMargin)
     val appendSink3 = util.createRetractTableSink(Array("a", "b"), Array(INT, LONG))
-    util.writeToSink(table5, appendSink3, "appendSink3")
+    util.tableEnv.registerTableSink("appendSink3", appendSink3)
+    stmtSet.addInsert("appendSink3", table5)
 
-    util.verifyExplain()
+    util.verifyExplain(stmtSet)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
index 68c50bb..a87c364 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/ModifiedMonotonicityTest.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.plan.stream.sql
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.functions.ScalarFunction
@@ -46,21 +47,21 @@ class ModifiedMonotonicityTest extends TableTestBase {
   def testMaxWithRetractOptimize(): Unit = {
     val query =
       "SELECT a1, MAX(a3) FROM (SELECT a1, a2, MAX(a3) AS a3 FROM A GROUP BY a1, a2) t GROUP BY a1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMinWithRetractOptimize(): Unit = {
     val query =
       "SELECT a1, MIN(a3) FROM (SELECT a1, a2, MIN(a3) AS a3 FROM A GROUP BY a1, a2) t GROUP BY a1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testMinCanNotOptimize(): Unit = {
     val query =
       "SELECT a1, MIN(a3) FROM (SELECT a1, a2, MAX(a3) AS a3 FROM A GROUP BY a1, a2) t GROUP BY a1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -71,7 +72,7 @@ class ModifiedMonotonicityTest extends TableTestBase {
       .setString(ExecutionConfigOptions.TABLE_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"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -81,7 +82,7 @@ class ModifiedMonotonicityTest extends TableTestBase {
     util.tableEnv.getConfig.getConfiguration
       .setString(ExecutionConfigOptions.TABLE_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)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -92,7 +93,7 @@ class ModifiedMonotonicityTest extends TableTestBase {
       .setString(ExecutionConfigOptions.TABLE_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"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala
index 007b4f3..4b1a038 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/PartitionableSinkTest.scala
@@ -32,7 +32,7 @@ class PartitionableSinkTest extends TableTestBase {
   createTable("sink", shuffleBy = false)
 
   private def createTable(name: String, shuffleBy: Boolean): Unit = {
-    util.tableEnv.sqlUpdate(
+    util.tableEnv.executeSql(
       s"""
          |create table $name (
          |  a bigint,
@@ -49,32 +49,32 @@ class PartitionableSinkTest extends TableTestBase {
 
   @Test
   def testStatic(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1, c=1) SELECT a FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (b=1, c=1) SELECT a FROM MyTable")
   }
 
   @Test
   def testDynamic(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink SELECT a, b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink SELECT a, b, c FROM MyTable")
   }
 
   @Test
   def testDynamicShuffleBy(): Unit = {
     createTable("sinkShuffleBy", shuffleBy = true)
-    util.verifySqlUpdate("INSERT INTO sinkShuffleBy SELECT a, b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sinkShuffleBy SELECT a, b, c FROM MyTable")
   }
 
   @Test
   def testPartial(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, c FROM MyTable")
   }
 
   @Test(expected = classOf[ValidationException])
   def testWrongStatic(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (a=1) SELECT b, c FROM MyTable")
   }
 
   @Test(expected = classOf[ValidationException])
   def testWrongFields(): Unit = {
-    util.verifySqlUpdate("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable")
+    util.verifyPlanInsert("INSERT INTO sink PARTITION (b=1) SELECT a, b, c FROM MyTable")
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala
index bcea370..d3a2906 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/RankTest.scala
@@ -19,7 +19,7 @@ package org.apache.flink.table.planner.plan.stream.sql
 
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.api.{ExplainDetail, TableException, ValidationException}
 import org.apache.flink.table.planner.utils.TableTestBase
 
 import org.junit.Test
@@ -279,7 +279,7 @@ class RankTest extends TableTestBase {
         |WHERE row_num <= 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -295,7 +295,7 @@ class RankTest extends TableTestBase {
         |WHERE 10 >= row_num
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -310,7 +310,7 @@ class RankTest extends TableTestBase {
         |WHERE row_num = 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -326,7 +326,7 @@ class RankTest extends TableTestBase {
         |WHERE row_num <= 10 AND b IS NOT NULL
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -348,7 +348,7 @@ class RankTest extends TableTestBase {
          |WHERE row_num <= 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -370,7 +370,7 @@ class RankTest extends TableTestBase {
          |WHERE row_num <= 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -390,7 +390,7 @@ class RankTest extends TableTestBase {
         |WHERE row_num <= 3
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -417,7 +417,7 @@ class RankTest extends TableTestBase {
          |SELECT max(a) FROM ($sql)
        """.stripMargin
 
-    util.verifyPlanWithTrait(sql2)
+    util.verifyPlan(sql2, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -440,7 +440,7 @@ class RankTest extends TableTestBase {
          |WHERE row_num <= 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -550,7 +550,7 @@ class RankTest extends TableTestBase {
          |WHERE row_num <= 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -575,7 +575,7 @@ class RankTest extends TableTestBase {
          |WHERE row_num <= 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -610,7 +610,7 @@ class RankTest extends TableTestBase {
          |WHERE rank_num <= 10
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test(expected = classOf[ValidationException])
@@ -636,7 +636,7 @@ class RankTest extends TableTestBase {
          |WHERE row_num <= a
       """.stripMargin
 
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   // TODO add tests about multi-sinks and udf
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SinkTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SinkTest.scala
index 1488081..f06ff8c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SinkTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SinkTest.scala
@@ -19,8 +19,8 @@
 package org.apache.flink.table.planner.plan.stream.sql
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.TableException
 import org.apache.flink.table.api.scala._
+import org.apache.flink.table.api.{ExplainDetail, TableException}
 import org.apache.flink.table.planner.utils.TableTestBase
 import org.apache.flink.table.types.logical.{BigIntType, IntType, VarCharType}
 
@@ -39,47 +39,47 @@ class SinkTest extends TableTestBase {
   def testExceptionForAppendSink(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT COUNT(*) AS cnt FROM MyTable GROUP BY a")
     val appendSink = util.createAppendTableSink(Array("a"), Array(LONG))
-    util.writeToSink(table, appendSink, "appendSink")
 
     thrown.expect(classOf[TableException])
     thrown.expectMessage("AppendStreamTableSink doesn't support consuming update " +
       "changes which is produced by node GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt])")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, appendSink, "appendSink")
   }
 
   @Test
   def testExceptionForOverAggregate(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val table = util.tableEnv.sqlQuery("SELECT COUNT(*) AS cnt FROM MyTable GROUP BY a")
     util.tableEnv.createTemporaryView("TempTable", table)
 
     val retractSink = util.createRetractTableSink(Array("cnt"), Array(LONG))
-    util.writeToSink(table, retractSink, "retractSink1")
+    util.tableEnv.registerTableSink("retractSink1", retractSink)
+    stmtSet.addInsert("retractSink1", table)
 
     val table2 = util.tableEnv.sqlQuery(
       "SELECT cnt, SUM(cnt) OVER (ORDER BY PROCTIME()) FROM TempTable")
     val retractSink2 = util.createRetractTableSink(Array("cnt", "total"), Array(LONG, LONG))
-    util.writeToSink(table2, retractSink2, "retractSink2")
+    util.tableEnv.registerTableSink("retractSink2", retractSink2)
+    stmtSet.addInsert("retractSink2", table2)
 
     thrown.expect(classOf[TableException])
     thrown.expectMessage("OverAggregate doesn't support consuming update changes " +
       "which is produced by node GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt])")
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet)
   }
 
   @Test
   def testAppendSink(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT a + b, c FROM MyTable")
     val appendSink = util.createAppendTableSink(Array("d", "c"), Array(LONG, STRING))
-    util.writeToSink(table, appendSink, "appendSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, appendSink, "appendSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testRetractSink1(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable GROUP BY a")
     val retractSink = util.createRetractTableSink(Array("a", "cnt"), Array(INT, LONG))
-    util.writeToSink(table, retractSink, "retractSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, retractSink, "retractSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -92,16 +92,14 @@ class SinkTest extends TableTestBase {
       """.stripMargin
     val table = util.tableEnv.sqlQuery(sqlQuery)
     val retractSink = util.createRetractTableSink(Array("cnt", "a"), Array(LONG, LONG))
-    util.writeToSink(table, retractSink, "retractSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, retractSink, "retractSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testUpsertSink1(): Unit = {
     val table = util.tableEnv.sqlQuery("SELECT a, COUNT(*) AS cnt FROM MyTable GROUP BY a")
     val upsertSink = util.createUpsertTableSink(Array(0), Array("a", "cnt"), Array(INT, LONG))
-    util.writeToSink(table, upsertSink, "upsertSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, upsertSink, "upsertSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -118,8 +116,7 @@ class SinkTest extends TableTestBase {
     val table = util.tableEnv.sqlQuery(sqlQuery)
     val upsertSink = util.createUpsertTableSink(Array(), Array("a1", "b", "c1"),
       Array(INT, LONG, STRING))
-    util.writeToSink(table, upsertSink, "upsertSink")
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, upsertSink, "upsertSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -132,46 +129,52 @@ class SinkTest extends TableTestBase {
         |""".stripMargin
     val table = util.tableEnv.sqlQuery(sql)
     val upsertSink = util.createUpsertTableSink(Array(0), Array("a", "cnt"), Array(INT, LONG))
-    util.writeToSink(table, upsertSink, "upsertSink")
     // a filter after aggregation, the Aggregation and Calc should produce UPDATE_BEFORE
-    util.verifyPlanWithTrait()
+    util.verifyPlanInsert(table, upsertSink, "upsertSink", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testRetractAndUpsertSink(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val table = util.tableEnv.sqlQuery("SELECT b, COUNT(a) AS cnt FROM MyTable GROUP BY b")
     util.tableEnv.registerTable("TempTable", table)
 
     val table1 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable WHERE b < 4")
     val retractSink = util.createRetractTableSink(Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table1, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable WHERE b >= 4 AND b < 6")
     val upsertSink = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table2, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table2)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testUpsertAndUpsertSink(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     val table = util.tableEnv.sqlQuery("SELECT b, COUNT(a) AS cnt FROM MyTable GROUP BY b")
     util.tableEnv.registerTable("TempTable", table)
 
     val table1 = util.tableEnv.sqlQuery(
       "SELECT cnt, COUNT(b) AS frequency FROM TempTable WHERE b < 4 GROUP BY cnt")
     val upsertSink1 = util.createUpsertTableSink(Array(0), Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table1, upsertSink1, "upsertSink1")
+    util.tableEnv.registerTableSink("upsertSink1", upsertSink1)
+    stmtSet.addInsert("upsertSink1", table1)
 
     val table2 = util.tableEnv.sqlQuery("SELECT b, cnt FROM TempTable WHERE b >= 4 AND b < 6")
     val upsertSink2 = util.createUpsertTableSink(Array(), Array("b", "cnt"), Array(LONG, LONG))
-    util.writeToSink(table2, upsertSink2,  "upsertSink2")
+    util.tableEnv.registerTableSink("upsertSink2", upsertSink2)
+    stmtSet.addInsert("upsertSink2", table2)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testAppendUpsertAndRetractSink(): Unit = {
+    val stmtSet = util.tableEnv.createStatementSet()
     util.addDataStream[(Int, Long, String)]("MyTable2", 'd, 'e, 'f)
     util.addDataStream[(Int, Long, String)]("MyTable3", 'i, 'j, 'k)
 
@@ -180,7 +183,8 @@ class SinkTest extends TableTestBase {
     util.tableEnv.registerTable("TempTable", table)
 
     val appendSink = util.createAppendTableSink(Array("a", "b"), Array(INT, LONG))
-    util.writeToSink(table, appendSink, "appendSink")
+    util.tableEnv.registerTableSink("appendSink", appendSink)
+    stmtSet.addInsert("appendSink", table)
 
     val table1 = util.tableEnv.sqlQuery(
       "SELECT a, b FROM TempTable UNION ALL SELECT i, j FROM MyTable3")
@@ -188,13 +192,15 @@ class SinkTest extends TableTestBase {
 
     val table2 = util.tableEnv.sqlQuery("SELECT SUM(a) AS total_sum FROM TempTable1")
     val retractSink = util.createRetractTableSink(Array("total_sum"), Array(INT))
-    util.writeToSink(table2, retractSink, "retractSink")
+    util.tableEnv.registerTableSink("retractSink", retractSink)
+    stmtSet.addInsert("retractSink", table2)
 
     val table3 = util.tableEnv.sqlQuery("SELECT MIN(a) AS total_min FROM TempTable1")
     val upsertSink = util.createUpsertTableSink(Array(), Array("total_min"), Array(INT))
-    util.writeToSink(table3, upsertSink, "upsertSink")
+    util.tableEnv.registerTableSink("upsertSink", upsertSink)
+    stmtSet.addInsert("upsertSink", table3)
 
-    util.verifyPlanWithTrait()
+    util.verifyPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala
index b7d7635..f509a5a 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/SortLimitTest.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.plan.stream.sql
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.utils.TableTestBase
 
@@ -77,12 +78,15 @@ class SortLimitTest extends TableTestBase {
 
   @Test
   def testOrderByLimit(): Unit = {
-    util.verifyPlanWithTrait("SELECT * FROM MyTable ORDER BY a, b DESC LIMIT 10")
+    util.verifyPlan(
+      "SELECT * FROM MyTable ORDER BY a, b DESC LIMIT 10", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testOrderByFetch(): Unit = {
-    util.verifyPlanWithTrait("SELECT * FROM MyTable ORDER BY a, b DESC FETCH FIRST 10 ROWS ONLY")
+    util.verifyPlan(
+      "SELECT * FROM MyTable ORDER BY a, b DESC FETCH FIRST 10 ROWS ONLY",
+      ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala
index c4a15b9..40fad2f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/TableScanTest.scala
@@ -19,7 +19,7 @@
 package org.apache.flink.table.planner.plan.stream.sql
 
 import org.apache.flink.api.scala._
-import org.apache.flink.table.api.{TableException, ValidationException}
+import org.apache.flink.table.api.{ExplainDetail, TableException, ValidationException}
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.expressions.utils.Func0
 import org.apache.flink.table.planner.factories.TestValuesTableFactory.{MockedFilterPushDownTableSource, MockedLookupTableSource}
@@ -152,7 +152,7 @@ class TableScanTest extends TableTestBase {
         |)
       """.stripMargin)
     // pass
-    util.verifyPlanWithTrait("SELECT * FROM src WHERE a > 1")
+    util.verifyPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -168,7 +168,7 @@ class TableScanTest extends TableTestBase {
         |  'changelog-mode' = 'I,UA,UB,D'
         |)
       """.stripMargin)
-    util.verifyPlanWithTrait("SELECT * FROM src WHERE a > 1")
+    util.verifyPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -184,7 +184,7 @@ class TableScanTest extends TableTestBase {
         |  'changelog-mode' = 'I,UA,UB'
         |)
       """.stripMargin)
-    util.verifyPlanWithTrait("SELECT COUNT(*) FROM src WHERE a > 1")
+    util.verifyPlan("SELECT COUNT(*) FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -204,7 +204,7 @@ class TableScanTest extends TableTestBase {
     thrown.expectMessage(
       "'default_catalog.default_database.src' source produces ChangelogMode " +
         "which contains UPDATE_BEFORE but doesn't contain UPDATE_AFTER, this is invalid.")
-    util.verifyPlanWithTrait("SELECT * FROM src WHERE a > 1")
+    util.verifyPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -224,7 +224,7 @@ class TableScanTest extends TableTestBase {
     thrown.expectMessage("Currently, ScanTableSource doesn't support producing " +
       "ChangelogMode which contains UPDATE_AFTER but no UPDATE_BEFORE. " +
       "Please adapt the implementation of 'TestValues' source.")
-    util.verifyPlanWithTrait("SELECT * FROM src WHERE a > 1")
+    util.verifyPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -244,7 +244,7 @@ class TableScanTest extends TableTestBase {
     thrown.expect(classOf[UnsupportedOperationException])
     thrown.expectMessage(
       "Currently, defining WATERMARK on a changelog source is not supported.")
-    util.verifyPlanWithTrait("SELECT * FROM src WHERE a > 1")
+    util.verifyPlan("SELECT * FROM src WHERE a > 1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -262,7 +262,7 @@ class TableScanTest extends TableTestBase {
       """.stripMargin)
     thrown.expect(classOf[TableException])
     thrown.expectMessage("Cannot generate a valid execution plan for the given query")
-    util.verifyPlanWithTrait("SELECT * FROM src")
+    util.verifyPlan("SELECT * FROM src", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -280,6 +280,6 @@ class TableScanTest extends TableTestBase {
       """.stripMargin)
     thrown.expect(classOf[UnsupportedOperationException])
     thrown.expectMessage("DynamicTableSource with SupportsFilterPushDown ability is not supported")
-    util.verifyPlanWithTrait("SELECT * FROM src")
+    util.verifyPlan("SELECT * FROM src", ExplainDetail.CHANGELOG_MODE)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
index 3addce3..730ece9 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/AggregateTest.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.scala._
 import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.api.scala._
-import org.apache.flink.table.api.{TableException, Types, ValidationException}
+import org.apache.flink.table.api.{ExplainDetail, TableException, Types, ValidationException}
 import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase}
 import org.apache.flink.table.runtime.typeutils.DecimalDataTypeInfo
 
@@ -185,7 +185,8 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testAvgWithRetract(): Unit = {
-    util.verifyPlanWithTrait("SELECT AVG(a) FROM (SELECT AVG(a) AS a FROM T GROUP BY b)")
+    util.verifyPlan(
+      "SELECT AVG(a) FROM (SELECT AVG(a) AS a FROM T GROUP BY b)", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -206,7 +207,8 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testSumWithRetract(): Unit = {
-    util.verifyPlanWithTrait("SELECT SUM(a) FROM (SELECT SUM(a) AS a FROM T GROUP BY b)")
+    util.verifyPlan(
+      "SELECT SUM(a) FROM (SELECT SUM(a) AS a FROM T GROUP BY b)", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -232,7 +234,8 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testMinWithRetract(): Unit = {
-    util.verifyPlanWithTrait("SELECT MIN(a) FROM (SELECT MIN(a) AS a FROM T GROUP BY b)")
+    util.verifyPlan(
+      "SELECT MIN(a) FROM (SELECT MIN(a) AS a FROM T GROUP BY b)", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -258,7 +261,8 @@ class AggregateTest extends TableTestBase {
 
   @Test
   def testMaxWithRetract(): Unit = {
-    util.verifyPlanWithTrait("SELECT MAX(a) FROM (SELECT MAX(a) AS a FROM T GROUP BY b)")
+    util.verifyPlan(
+      "SELECT MAX(a) FROM (SELECT MAX(a) AS a FROM T GROUP BY b)", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
index 276d833..b698408 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/DistinctAggregateTest.scala
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.stream.sql.agg
 
 import org.apache.flink.api.common.time.Time
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.config.OptimizerConfigOptions
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.plan.rules.physical.stream.IncrementalAggregateRule
@@ -165,7 +166,7 @@ class DistinctAggregateTest(
         |  GROUP BY c
         |) GROUP BY a
       """.stripMargin
-    util.verifyPlanWithTrait(sqlQuery)
+    util.verifyPlan(sqlQuery, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -181,7 +182,7 @@ class DistinctAggregateTest(
          |  GROUP BY a
          |) GROUP BY c
        """.stripMargin
-    util.verifyPlanWithTrait(sqlQuery)
+    util.verifyPlan(sqlQuery, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -197,7 +198,7 @@ class DistinctAggregateTest(
          |  GROUP BY a
          |) GROUP BY b
        """.stripMargin
-    util.verifyPlanWithTrait(sqlQuery)
+    util.verifyPlan(sqlQuery, ExplainDetail.CHANGELOG_MODE)
   }
 }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala
index fe66dd4..f219c46 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/agg/WindowAggregateTest.scala
@@ -21,7 +21,7 @@ package org.apache.flink.table.planner.plan.stream.sql.agg
 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.{TableException, ValidationException}
+import org.apache.flink.table.api.{ExplainDetail, TableException, ValidationException}
 import org.apache.flink.table.planner.plan.utils.JavaUserDefinedAggFunctions.WeightedAvgWithMerge
 import org.apache.flink.table.planner.plan.utils.WindowEmitStrategy.{TABLE_EXEC_EMIT_LATE_FIRE_DELAY, TABLE_EXEC_EMIT_LATE_FIRE_ENABLED}
 import org.apache.flink.table.planner.utils.TableTestBase
@@ -35,7 +35,7 @@ class WindowAggregateTest extends TableTestBase {
   util.addDataStream[(Int, String, Long)](
     "MyTable", 'a, 'b, 'c, 'proctime.proctime, 'rowtime.rowtime)
   util.addFunction("weightedAvg", new WeightedAvgWithMerge)
-  util.tableEnv.sqlUpdate(
+  util.tableEnv.executeSql(
     s"""
        |create table MyTable1 (
        |  a int,
@@ -428,7 +428,7 @@ class WindowAggregateTest extends TableTestBase {
         |FROM MyTable
         |GROUP BY TUMBLE(`rowtime`, INTERVAL '1' SECOND)
         |""".stripMargin
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -440,6 +440,6 @@ class WindowAggregateTest extends TableTestBase {
         |FROM MyTable
         |GROUP BY TUMBLE(`rowtime`, INTERVAL '1' SECOND)
         |""".stripMargin
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala
index ee386b3..96e103c 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/JoinTest.scala
@@ -19,6 +19,7 @@
 package org.apache.flink.table.planner.plan.stream.sql.join
 
 import org.apache.flink.api.scala._
+import org.apache.flink.table.api.ExplainDetail
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.planner.utils.{StreamTableTestUtil, TableTestBase}
 
@@ -42,7 +43,7 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, b1 FROM ($query1) JOIN ($query2) ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -50,12 +51,13 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, a2, b1, b2 FROM ($query1) JOIN ($query2) ON a2 = b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testLeftJoinNonEqui(): Unit = {
-    util.verifyPlanWithTrait("SELECT a1, b1 FROM A LEFT JOIN B ON a1 = b1 AND a2 > b2")
+    util.verifyPlan(
+      "SELECT a1, b1 FROM A LEFT JOIN B ON a1 = b1 AND a2 > b2", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -63,14 +65,14 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, b1 FROM ($query1) LEFT JOIN ($query2) ON a1 = b1 AND a2 > b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testLeftJoinWithRightNotPkNonEqui(): Unit = {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query = s"SELECT a1, b1 FROM ($query1) LEFT JOIN B ON a1 = b1 AND a2 > b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -78,12 +80,12 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, a2, b1, b2 FROM ($query1) LEFT JOIN ($query2) ON a2 = b2 AND a1 > b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testLeftJoin(): Unit = {
-    util.verifyPlanWithTrait("SELECT a1, b1 FROM A LEFT JOIN B ON a1 = b1")
+    util.verifyPlan("SELECT a1, b1 FROM A LEFT JOIN B ON a1 = b1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -91,14 +93,14 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, b1 FROM ($query1) LEFT JOIN ($query2) ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testLeftJoinWithRightNotPk(): Unit = {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query = s"SELECT a1, b1 FROM ($query1) LEFT JOIN B ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -106,12 +108,13 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, a2, b1, b2 FROM ($query1) LEFT JOIN ($query2) ON a2 = b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testRightJoinNonEqui(): Unit = {
-    util.verifyPlanWithTrait("SELECT a1, b1 FROM A RIGHT JOIN B ON a1 = b1 AND a2 > b2")
+    util.verifyPlan(
+      "SELECT a1, b1 FROM A RIGHT JOIN B ON a1 = b1 AND a2 > b2", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -119,14 +122,14 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, b1 FROM ($query1) RIGHT JOIN ($query2) ON a1 = b1 AND a2 > b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testRightJoinWithRightNotPkNonEqui(): Unit = {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query = s"SELECT a1, b1 FROM ($query1) RIGHT JOIN B ON a1 = b1 AND a2 > b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -134,12 +137,12 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, a2, b1, b2 FROM ($query1) RIGHT JOIN ($query2) ON a2 = b2 AND a1 > b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testRightJoin(): Unit = {
-    util.verifyPlanWithTrait("SELECT a1, b1 FROM A RIGHT JOIN B ON a1 = b1")
+    util.verifyPlan("SELECT a1, b1 FROM A RIGHT JOIN B ON a1 = b1", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -147,14 +150,14 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, b1 FROM ($query1) RIGHT JOIN ($query2) ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testRightJoinWithRightNotPk(): Unit = {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A group by a1"
     val query = s"SELECT a1, b1 FROM ($query1) RIGHT JOIN B ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -162,12 +165,13 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A group by a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B group by b1"
     val query = s"SELECT a1, a2, b1, b2 FROM ($query1) RIGHT JOIN ($query2) ON a2 = b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testFullJoinNonEqui(): Unit = {
-    util.verifyPlanWithTrait("SELECT a1, b1 FROM A FULL JOIN B ON a1 = b1 AND a2 > b2")
+    util.verifyPlan(
+      "SELECT a1, b1 FROM A FULL JOIN B ON a1 = b1 AND a2 > b2", ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -175,14 +179,14 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, b1 FROM ($query1) FULL JOIN ($query2) ON a1 = b1 AND a2 > b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testFullJoinWithFullNotPkNonEqui(): Unit = {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query = s"SELECT a1, b1 FROM ($query1) FULL JOIN B ON a1 = b1 AND a2 > b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -190,13 +194,13 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, a2, b1, b2 FROM ($query1) FULL JOIN ($query2) ON a2 = b2 AND a1 > b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testFullJoin(): Unit = {
     val query = "SELECT a1, b1 FROM A FULL JOIN B ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -204,14 +208,14 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, b1 FROM ($query1) FULL JOIN ($query2) ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
   def testFullJoinWithFullNotPk(): Unit = {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query = s"SELECT a1, b1 FROM ($query1) FULL JOIN B ON a1 = b1"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -219,7 +223,7 @@ class JoinTest extends TableTestBase {
     val query1 = "SELECT SUM(a2) AS a2, a1 FROM A GROUP BY a1"
     val query2 = "SELECT SUM(b2) AS b2, b1 FROM B GROUP BY b1"
     val query = s"SELECT a1, a2, b1, b2 FROM ($query1) FULL JOIN ($query2) ON a2 = b2"
-    util.verifyPlanWithTrait(query)
+    util.verifyPlan(query, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
@@ -233,7 +237,7 @@ class JoinTest extends TableTestBase {
          |  SELECT * FROM src WHERE key = 0) src2
          |ON (src1.key = src2.key AND src2.key > 10)
        """.stripMargin
-    util.verifyPlanWithTrait(sql)
+    util.verifyPlan(sql, ExplainDetail.CHANGELOG_MODE)
   }
 
   @Test
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala
index 7b2159a..6a3cfa2 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/join/LookupJoinTest.scala
@@ -470,7 +470,7 @@ class LookupJoinTest(legacyTableSource: Boolean) extends TableTestBase with Seri
   }
 
   private def verifyTranslationSuccess(sql: String): Unit = {
-    util.tableEnv.explain(util.tableEnv.sqlQuery(sql))
+    util.tableEnv.sqlQuery(sql).explain()
   }
 }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala
index 5fd3a5c..cec7edf 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/TableSourceTest.scala
@@ -46,7 +46,7 @@ class TableSourceTest extends TableTestBase {
       new TestTableSourceWithTime[Row](
         false, tableSchema, returnType, Seq(), rowtime = "rowtime"))
 
-    val t = util.tableEnv.scan("rowTimeT").select($"rowtime", $"id", $"name", $"val")
+    val t = util.tableEnv.from("rowTimeT").select($"rowtime", $"id", $"name", $"val")
     util.verifyPlan(t)
   }
 
@@ -67,7 +67,7 @@ class TableSourceTest extends TableTestBase {
       new TestTableSourceWithTime[Row](
         false, tableSchema, returnType, Seq(), rowtime = "rowtime"))
 
-    val t = util.tableEnv.scan("rowTimeT").select($"rowtime", $"id", $"name", $"val")
+    val t = util.tableEnv.from("rowTimeT").select($"rowtime", $"id", $"name", $"val")
     util.verifyPlan(t)
   }
 
@@ -88,7 +88,7 @@ class TableSourceTest extends TableTestBase {
       new TestTableSourceWithTime[Row](
         false, tableSchema, returnType, Seq(), rowtime = "rowtime"))
 
-    val t = util.tableEnv.scan("rowTimeT")
+    val t = util.tableEnv.from("rowTimeT")
       .where($"val" > 100)
       .window(Tumble over 10.minutes on 'rowtime as 'w)
       .groupBy('name, 'w)
@@ -112,7 +112,7 @@ class TableSourceTest extends TableTestBase {
       new TestTableSourceWithTime[Row](
         false, tableSchema, returnType, Seq(), proctime = "proctime"))
 
-    val t = util.tableEnv.scan("procTimeT").select($"proctime", $"id", $"name", $"val")
+    val t = util.tableEnv.from("procTimeT").select($"proctime", $"id", $"name", $"val")
     util.verifyPlan(t)
   }
 
@@ -132,7 +132,7 @@ class TableSourceTest extends TableTestBase {
       new TestTableSourceWithTime[Row](
         false, tableSchema, returnType, Seq(), proctime = "proctime"))
 
-    val t = util.tableEnv.scan("procTimeT")
+    val t = util.tableEnv.from("procTimeT")
       .window(Over partitionBy 'id orderBy 'proctime preceding 2.hours as 'w)
       .select('id, 'name, 'val.sum over 'w as 'valSum)
       .filter('valSum > 100)
@@ -155,7 +155,7 @@ class TableSourceTest extends TableTestBase {
       new TestProjectableTableSource(
         false, tableSchema, returnType, Seq(), "rtime", "ptime"))
 
-    val t = util.tableEnv.scan("T").select('name, 'val, 'id)
+    val t = util.tableEnv.from("T").select('name, 'val, 'id)
     util.verifyPlan(t)
   }
 
@@ -175,7 +175,7 @@ class TableSourceTest extends TableTestBase {
       new TestProjectableTableSource(
         false, tableSchema, returnType, Seq(), "rtime", "ptime"))
 
-    val t = util.tableEnv.scan("T").select('ptime, 'name, 'val, 'id)
+    val t = util.tableEnv.from("T").select('ptime, 'name, 'val, 'id)
     util.verifyPlan(t)
   }
 
@@ -194,7 +194,7 @@ class TableSourceTest extends TableTestBase {
       new TestProjectableTableSource(
         false, tableSchema, returnType, Seq(), "rtime", "ptime"))
 
-    val t = util.tableEnv.scan("T").select('name, 'val, 'rtime, 'id)
+    val t = util.tableEnv.from("T").select('name, 'val, 'rtime, 'id)
     util.verifyPlan(t)
   }
 
@@ -213,7 +213,7 @@ class TableSourceTest extends TableTestBase {
       new TestProjectableTableSource(
         false, tableSchema, returnType, Seq(), "rtime", "ptime"))
 
-    val t = util.tableEnv.scan("T").select('ptime)
+    val t = util.tableEnv.from("T").select('ptime)
     util.verifyPlan(t)
   }
 
@@ -232,7 +232,7 @@ class TableSourceTest extends TableTestBase {
       new TestProjectableTableSource(
         false, tableSchema, returnType, Seq(), "rtime", "ptime"))
 
-    val t = util.tableEnv.scan("T").select('rtime)
+    val t = util.tableEnv.from("T").select('rtime)
     util.verifyPlan(t)
   }
 
@@ -253,7 +253,7 @@ class TableSourceTest extends TableTestBase {
       new TestProjectableTableSource(
         false, tableSchema, returnType, Seq(), "rtime", "ptime", mapping))
 
-    val t = util.tableEnv.scan("T").select('name, 'rtime, 'val)
+    val t = util.tableEnv.from("T").select('name, 'rtime, 'val)
     util.verifyPlan(t)
   }
 
@@ -290,7 +290,7 @@ class TableSourceTest extends TableTestBase {
         false, tableSchema, returnType, Seq()))
 
     val t = util.tableEnv
-      .scan("T")
+      .from("T")
       .select('id,
         'deepNested.get("nested1").get("name") as 'nestedName,
         'nested.get("value") as 'nestedValue,
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala
index f91de9a..77cd239 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/stream/table/validation/TableSinkValidationTest.scala
@@ -23,9 +23,10 @@ import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{DataTypes, TableException, TableSchema, ValidationException}
-import org.apache.flink.table.planner.runtime.utils.{TestData, TestingAppendSink, TestingUpsertTableSink}
+import org.apache.flink.table.planner.runtime.utils.{TableEnvUtil, TestData, TestingAppendSink, TestingUpsertTableSink}
 import org.apache.flink.table.planner.utils.{MemoryTableSourceSinkUtil, TableTestBase, TableTestUtil}
 import org.apache.flink.types.Row
+
 import org.junit.Test
 
 class TableSinkValidationTest extends TableTestBase {
@@ -109,10 +110,8 @@ class TableSinkValidationTest extends TableTestBase {
 
     MemoryTableSourceSinkUtil.createDataTypeOutputFormatTable(
       tEnv, sinkSchema, "testSink")
-    tEnv.insertInto(resultTable, "testSink")
-
     // must fail because query result table schema is different with sink table schema
-    tEnv.execute("testJob")
+    TableEnvUtil.execInsertTableAndWaitResult(resultTable, "testSink")
   }
 
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala
index f303228..3b7fd46 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/FileSystemITCaseBase.scala
@@ -25,6 +25,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions
 import org.apache.flink.table.planner.runtime.FileSystemITCaseBase._
 import org.apache.flink.table.planner.runtime.utils.BatchTableEnvUtil
 import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row
+import org.apache.flink.table.planner.runtime.utils.TableEnvUtil.execInsertSqlAndWaitResult
 import org.apache.flink.types.Row
 
 import org.junit.rules.TemporaryFolder
@@ -62,7 +63,7 @@ trait FileSystemITCaseBase {
       data_with_partitions,
       dataType,
       "x, y, a, b")
-    tableEnv.sqlUpdate(
+    tableEnv.executeSql(
       s"""
          |create table partitionedTable (
          |  x string,
@@ -76,7 +77,7 @@ trait FileSystemITCaseBase {
          |)
        """.stripMargin
     )
-    tableEnv.sqlUpdate(
+    tableEnv.executeSql(
       s"""
          |create table nonPartitionedTable (
          |  x string,
@@ -94,9 +95,8 @@ trait FileSystemITCaseBase {
 
   @Test
   def testAllStaticPartitions1(): Unit = {
-    tableEnv.sqlUpdate("insert into partitionedTable " +
+    execInsertSqlAndWaitResult(tableEnv, "insert into partitionedTable " +
         "partition(a='1', b='1') select x, y from originalT where a=1 and b=1")
-    tableEnv.execute("test")
 
     check(
       "select x, y from partitionedTable where a=1 and b=1",
@@ -111,9 +111,8 @@ trait FileSystemITCaseBase {
 
   @Test
   def testAllStaticPartitions2(): Unit = {
-    tableEnv.sqlUpdate("insert into partitionedTable " +
+    execInsertSqlAndWaitResult(tableEnv, "insert into partitionedTable " +
         "partition(a='2', b='1') select x, y from originalT where a=2 and b=1")
-    tableEnv.execute("test")
 
     check(
       "select x, y from partitionedTable where a=2 and b=1",
@@ -128,9 +127,8 @@ trait FileSystemITCaseBase {
 
   @Test
   def testPartialDynamicPartition(): Unit = {
-    tableEnv.sqlUpdate("insert into partitionedTable " +
+    execInsertSqlAndWaitResult(tableEnv, "insert into partitionedTable " +
         "partition(a=3) select x, y, b from originalT where a=3")
-    tableEnv.execute("test")
 
     check(
       "select x, y from partitionedTable where a=2 and b=1",
@@ -170,9 +168,8 @@ trait FileSystemITCaseBase {
 
   @Test
   def testDynamicPartition(): Unit = {
-    tableEnv.sqlUpdate("insert into partitionedTable " +
+    execInsertSqlAndWaitResult(tableEnv, "insert into partitionedTable " +
         "select x, y, a, b from originalT")
-    tableEnv.execute("test")
 
     check(
       "select x, y from partitionedTable where a=1 and b=1",
@@ -192,9 +189,8 @@ trait FileSystemITCaseBase {
 
   @Test
   def testNonPartition(): Unit = {
-    tableEnv.sqlUpdate("insert into nonPartitionedTable " +
+    execInsertSqlAndWaitResult(tableEnv, "insert into nonPartitionedTable " +
         "select x, y, a, b from originalT where a=1 and b=1")
-    tableEnv.execute("test")
 
     check(
       "select x, y from nonPartitionedTable where a=1 and b=1",
@@ -206,8 +202,8 @@ trait FileSystemITCaseBase {
   def testLimitPushDown(): Unit = {
     tableEnv.getConfig.getConfiguration.setInteger(
       ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1)
-    tableEnv.sqlUpdate("insert into nonPartitionedTable select x, y, a, b from originalT")
-    tableEnv.execute("test")
+    execInsertSqlAndWaitResult(
+      tableEnv, "insert into nonPartitionedTable select x, y, a, b from originalT")
 
     check(
       "select x, y from nonPartitionedTable limit 3",
@@ -219,8 +215,8 @@ trait FileSystemITCaseBase {
 
   @Test
   def testFilterPushDown(): Unit = {
-    tableEnv.sqlUpdate("insert into nonPartitionedTable select x, y, a, b from originalT")
-    tableEnv.execute("test")
+    execInsertSqlAndWaitResult(
+      tableEnv, "insert into nonPartitionedTable select x, y, a, b from originalT")
 
     check(
       "select x, y from nonPartitionedTable where a=10086",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala
index 8c33f98..783e152 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/LimitITCase.scala
@@ -103,7 +103,7 @@ class LimitITCase extends BatchTestBase {
   @Test(expected = classOf[ValidationException])
   def testTableLimitWithLimitTable(): Unit = {
     Assert.assertEquals(
-      executeQuery(tEnv.scan("LimitTable").fetch(5)).size,
+      executeQuery(tEnv.from("LimitTable").fetch(5)).size,
       5)
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
index 19b3bde..a27385b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala
@@ -33,7 +33,7 @@ import org.apache.flink.table.descriptors.Schema.SCHEMA
 import org.apache.flink.table.factories.TableSinkFactory
 import org.apache.flink.table.filesystem.FileSystemTableFactory
 import org.apache.flink.table.planner.runtime.batch.sql.PartitionableSinkITCase._
-import org.apache.flink.table.planner.runtime.utils.BatchTestBase
+import org.apache.flink.table.planner.runtime.utils.{BatchTestBase, TableEnvUtil}
 import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.planner.runtime.utils.TestData._
 import org.apache.flink.table.sinks.{PartitionableTableSink, StreamTableSink, TableSink}
@@ -77,9 +77,8 @@ class PartitionableSinkITCase extends BatchTestBase {
   @Test
   def testInsertWithOutPartitionGrouping(): Unit = {
     registerTableSink()
-    tEnv.sqlUpdate("insert into sinkTable select a, max(b), c"
+    execInsertSqlAndWaitResult("insert into sinkTable select a, max(b), c"
       + " from nonSortTable group by a, c")
-    tEnv.execute("testJob")
     assertEquals(List("1,5,Hi",
       "1,5,Hi01",
       "1,5,Hi02"),
@@ -102,8 +101,7 @@ class PartitionableSinkITCase extends BatchTestBase {
   @Test
   def testInsertWithPartitionGrouping(): Unit = {
     registerTableSink()
-    tEnv.sqlUpdate("insert into sinkTable select a, b, c from sortTable")
-    tEnv.execute("testJob")
+    execInsertSqlAndWaitResult("insert into sinkTable select a, b, c from sortTable")
     assertEquals(List("1,1,Hello world",
       "1,1,Hello world, how are you?"),
       RESULT1.toList)
@@ -123,8 +121,7 @@ class PartitionableSinkITCase extends BatchTestBase {
   @Test
   def testInsertWithStaticPartitions(): Unit = {
     registerTableSink()
-    tEnv.sqlUpdate("insert into sinkTable partition(a=1) select b, c from sortTable")
-    tEnv.execute("testJob")
+    execInsertSqlAndWaitResult("insert into sinkTable partition(a=1) select b, c from sortTable")
     assertEquals(List("1,2,Hi",
       "1,1,Hello world",
       "1,2,Hello",
@@ -144,8 +141,7 @@ class PartitionableSinkITCase extends BatchTestBase {
   @Test
   def testInsertWithStaticAndDynamicPartitions(): Unit = {
     registerTableSink(partitionColumns = Array("a", "b"))
-    tEnv.sqlUpdate("insert into sinkTable partition(a=1) select b, c from sortTable")
-    tEnv.execute("testJob")
+    execInsertSqlAndWaitResult("insert into sinkTable partition(a=1) select b, c from sortTable")
     assertEquals(List("1,1,Hello world", "1,1,Hello world, how are you?"), RESULT1.toList)
     assertEquals(List(
       "1,4,你好,陌生人",
@@ -167,16 +163,14 @@ class PartitionableSinkITCase extends BatchTestBase {
     expectedEx.expect(classOf[ValidationException])
     registerTableSink(tableName = "sinkTable2", rowType = type4,
       partitionColumns = Array("a", "b"))
-    tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sortTable")
-    tEnv.execute("testJob")
+    execInsertSqlAndWaitResult("insert into sinkTable2 partition(c=1) select a, b from sortTable")
   }
 
   @Test
   def testInsertStaticPartitionOnNonPartitionedSink(): Unit = {
     expectedEx.expect(classOf[TableException])
     registerTableSink(tableName = "sinkTable2", rowType = type4, partitionColumns = Array())
-    tEnv.sqlUpdate("insert into sinkTable2 partition(c=1) select a, b from sortTable")
-    tEnv.execute("testJob")
+    execInsertSqlAndWaitResult("insert into sinkTable2 partition(c=1) select a, b from sortTable")
   }
 
   private def registerTableSink(
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
index 662f77f..ceb5797 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/join/JoinITCase.scala
@@ -20,25 +20,26 @@ package org.apache.flink.table.planner.runtime.batch.sql.join
 
 import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{DOUBLE_TYPE_INFO, INT_TYPE_INFO, LONG_TYPE_INFO}
+import org.apache.flink.api.common.typeinfo.Types
 import org.apache.flink.api.common.typeutils.TypeComparator
+import org.apache.flink.api.dag.Transformation
 import org.apache.flink.api.java.typeutils.{GenericTypeInfo, RowTypeInfo}
+import org.apache.flink.streaming.api.transformations.{OneInputTransformation, SinkTransformation, TwoInputTransformation}
+import org.apache.flink.table.planner.delegation.PlannerBase
 import org.apache.flink.table.planner.expressions.utils.FuncWithOpen
 import org.apache.flink.table.planner.runtime.batch.sql.join.JoinType.{BroadcastHashJoin, HashJoin, JoinType, NestedLoopJoin, SortMergeJoin}
 import org.apache.flink.table.planner.runtime.utils.BatchTestBase
 import org.apache.flink.table.planner.runtime.utils.BatchTestBase.row
 import org.apache.flink.table.planner.runtime.utils.TestData._
+import org.apache.flink.table.planner.sinks.CollectRowTableSink
+import org.apache.flink.table.planner.utils.{TestingStatementSet, TestingTableEnvironment}
+import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+
 import org.junit.runner.RunWith
 import org.junit.runners.Parameterized
 import org.junit.{Assert, Before, Test}
-import java.util
 
-import org.apache.flink.api.common.typeinfo.Types
-import org.apache.flink.api.dag.Transformation
-import org.apache.flink.streaming.api.transformations.{OneInputTransformation, SinkTransformation, TwoInputTransformation}
-import org.apache.flink.table.planner.delegation.PlannerBase
-import org.apache.flink.table.planner.sinks.CollectRowTableSink
-import org.apache.flink.table.planner.utils.TestingTableEnvironment
-import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory
+import java.util
 
 import scala.collection.JavaConversions._
 import scala.collection.Seq
@@ -101,10 +102,13 @@ class JoinITCase(expectedJoinType: JoinType) extends BatchTestBase {
     if (expectedJoinType == HashJoin) {
       val sink = (new CollectRowTableSink).configure(Array("c"), Array(Types.STRING))
       tEnv.registerTableSink("outputTable", sink)
-      tEnv.insertInto("outputTable", tEnv.sqlQuery("SELECT c FROM SmallTable3, Table5 WHERE b = e"))
+      val stmtSet = tEnv.createStatementSet()
+      val table = tEnv.sqlQuery("SELECT c FROM SmallTable3, Table5 WHERE b = e")
+      stmtSet.addInsert("outputTable", table)
       val testingTEnv = tEnv.asInstanceOf[TestingTableEnvironment]
+      val testingStmtSet = stmtSet.asInstanceOf[TestingStatementSet]
       val transforms = testingTEnv.getPlanner.asInstanceOf[PlannerBase]
-        .translate(testingTEnv.getBufferedOperations)
+        .translate(testingStmtSet.getOperations)
       var haveTwoOp = false
 
       @scala.annotation.tailrec
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/OverWindowITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/OverWindowITCase.scala
index d2bddb1..97a2bc5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/OverWindowITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/OverWindowITCase.scala
@@ -52,7 +52,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testSingleRowOverWindow(): Unit = {
-    val table = tEnv.scan("Table1")
+    val table = tEnv.from("Table1")
 
     val expected = Seq(
       row("a", 1, 5),
@@ -97,7 +97,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testUnboundedRowOverWindow(): Unit = {
-    val table = tEnv.scan("Table1")
+    val table = tEnv.from("Table1")
 
     checkTableResult(
       table
@@ -119,7 +119,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testOrderByStringTypeField(): Unit = {
-    val table2 = tEnv.scan("Table2")
+    val table2 = tEnv.from("Table2")
 
     checkTableResult(
       table2
@@ -174,7 +174,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationSumWithOrderBy(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -204,7 +204,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationSumWithOrderByAndWithoutPartitionBy(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -234,7 +234,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationCountWithOrderBy(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -264,7 +264,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationAvgWithOrderBy(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -294,7 +294,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationSumWithOrderByWithRowsBetween(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -324,7 +324,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationSumWithOrderByShrinkWindow(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     val expected = Seq(
       row(1, 1, 1),
@@ -370,7 +370,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationSumWithOrderByWithRangeBetween(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -417,7 +417,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationWithOrderByWithRowBetween(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -462,7 +462,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationMaxWithOrderBy(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -492,7 +492,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationMinWithOrderBy(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -522,7 +522,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testAvg(): Unit = {
-    val table = tEnv.scan("Table4")
+    val table = tEnv.from("Table4")
 
     checkTableResult(
       table
@@ -545,7 +545,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testRangeFrame(): Unit = {
-    val table = tEnv.scan("Table4")
+    val table = tEnv.from("Table4")
 
     // sliding frame case: 1 - 1
     checkTableResult(
@@ -607,7 +607,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testRowsFrame(): Unit = {
-    val table = tEnv.scan("Table4")
+    val table = tEnv.from("Table4")
 
     // sliding frame case: unbounded - 1
     checkTableResult(
@@ -650,7 +650,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testRangeFrameWithNullValue(): Unit = {
-    val table = tEnv.scan("NullTable4")
+    val table = tEnv.from("NullTable4")
 
     checkTableResult(
       table
@@ -673,7 +673,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testNegativeRows(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -840,7 +840,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testNegativeRange(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -1007,7 +1007,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testFractionalRange(): Unit = {
-    val table = tEnv.scan("Table5")
+    val table = tEnv.from("Table5")
 
     checkTableResult(
       table
@@ -1062,7 +1062,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggWithConstants(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
@@ -1142,7 +1142,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggWithNull(): Unit = {
-    val table = tEnv.scan("NullTable2")
+    val table = tEnv.from("NullTable2")
 
     checkTableResult(
       table
@@ -1234,7 +1234,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationAtDate(): Unit = {
-    val table = tEnv.scan("Table3")
+    val table = tEnv.from("Table3")
 
     checkTableResult(
       table
@@ -1266,7 +1266,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testWindowAggregationAtTime(): Unit = {
-    val table = tEnv.scan("Table3")
+    val table = tEnv.from("Table3")
 
     checkTableResult(
       table
@@ -1298,7 +1298,7 @@ class OverWindowITCase extends BatchTestBase {
 
   @Test
   def testBatchCurrentRange(): Unit = {
-    val table = tEnv.scan("Table2")
+    val table = tEnv.from("Table2")
 
     checkTableResult(
       table
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala
index 39485a9..14224e6 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/table/TableSinkITCase.scala
@@ -20,8 +20,8 @@ package org.apache.flink.table.planner.runtime.batch.table
 
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{DataTypes, TableSchema}
-import org.apache.flink.table.planner.runtime.utils.{BatchTestBase, TestingRetractTableSink, TestingUpsertTableSink}
 import org.apache.flink.table.planner.runtime.utils.TestData._
+import org.apache.flink.table.planner.runtime.utils.{BatchTestBase, TestingRetractTableSink, TestingUpsertTableSink}
 import org.apache.flink.table.planner.utils.MemoryTableSourceSinkUtil
 import org.apache.flink.test.util.TestBaseUtils
 
@@ -48,12 +48,10 @@ class TableSinkITCase extends BatchTestBase {
       tEnv, schema, "testSink")
     registerCollection("Table3", data3, type3, "a, b, c", nullablesOfData3)
 
-    tEnv.scan("Table3")
+    val table = tEnv.from("Table3")
         .where('a > 20)
         .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5)))
-        .insertInto("testSink")
-
-    tEnv.execute("")
+    execInsertTableAndWaitResult(table, "testSink")
 
     val results = MemoryTableSourceSinkUtil.tableDataStrings.asJava
     val expected = Seq("12345,55,12345").mkString("\n")
@@ -76,12 +74,10 @@ class TableSinkITCase extends BatchTestBase {
 
     registerCollection("Table3", data3, type3, "a, b, c", nullablesOfData3)
 
-    tEnv.scan("Table3")
+    val table = tEnv.from("Table3")
         .where('a > 20)
         .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5)))
-        .insertInto("testSink")
-
-    tEnv.execute("")
+    execInsertTableAndWaitResult(table, "testSink")
 
     val results = MemoryTableSourceSinkUtil.tableDataStrings.asJava
     val expected = Seq("12345,55,12345").mkString("\n")
@@ -106,12 +102,10 @@ class TableSinkITCase extends BatchTestBase {
 
     registerCollection("Table3", simpleData2, simpleType2, "a, b", nullableOfSimpleData2)
 
-    tEnv.from("Table3")
+    val table = tEnv.from("Table3")
       .select('a.cast(DataTypes.STRING()), 'b.cast(DataTypes.DECIMAL(10, 2)))
       .distinct()
-      .insertInto("testSink")
-
-    tEnv.execute("")
+    execInsertTableAndWaitResult(table, "testSink")
 
     val results = MemoryTableSourceSinkUtil.tableDataStrings.asJava
     val expected = Seq("1,0.100000000000000000", "2,0.200000000000000000",
@@ -139,11 +133,10 @@ class TableSinkITCase extends BatchTestBase {
     sink.expectedKeys = Some(Array("a"))
     sink.expectedIsAppendOnly = Some(false)
 
-    tEnv.from("MyTable")
+   val table = tEnv.from("MyTable")
         .groupBy('a)
         .select('a, 'b.sum())
-        .insertInto("testSink")
-    tEnv.execute("")
+    execInsertTableAndWaitResult(table, "testSink")
 
     val result = sink.getUpsertResults.sorted
     val expected = List(
@@ -161,11 +154,10 @@ class TableSinkITCase extends BatchTestBase {
     sink.expectedKeys = None
     sink.expectedIsAppendOnly = Some(true)
 
-    tEnv.from("MyTable")
+    val table = tEnv.from("MyTable")
         .select('a, 'b)
         .where('a < 3)
-        .insertInto("testSink")
-    tEnv.execute("")
+    execInsertTableAndWaitResult(table, "testSink")
 
     val result = sink.getRawResults.sorted
     val expected = List(
@@ -190,11 +182,10 @@ class TableSinkITCase extends BatchTestBase {
   def testRetractSink(): Unit = {
     val sink = prepareForRetractSink()
 
-    tEnv.from("MyTable")
+    val table = tEnv.from("MyTable")
         .groupBy('a)
         .select('a, 'b.sum())
-        .insertInto("testSink")
-    tEnv.execute("")
+    execInsertTableAndWaitResult(table, "testSink")
 
     val result = sink.getRawResults.sorted
     val expected = List(
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala
index acfce75..7c1ab41 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/FsStreamingSinkITCaseBase.scala
@@ -95,10 +95,12 @@ abstract class FsStreamingSinkITCaseBase extends StreamingTestBase {
                  |  ${additionalProperties().mkString(",\n")}
                  |)
        """.stripMargin
-    tEnv.sqlUpdate(ddl)
+    tEnv.executeSql(ddl)
 
-    tEnv.insertInto("sink_table", tEnv.sqlQuery("select * from my_table"))
-    tEnv.execute("insert")
+    val tableResult = tEnv.sqlQuery("select * from my_table").executeInsert("sink_table")
+    tableResult.getJobClient.get()
+      .getJobExecutionResult(Thread.currentThread().getContextClassLoader)
+      .get()
 
     check(
       ddl,
@@ -109,7 +111,7 @@ abstract class FsStreamingSinkITCaseBase extends StreamingTestBase {
   def check(ddl: String, sqlQuery: String, expectedResult: Seq[Row]): Unit = {
     val setting = EnvironmentSettings.newInstance().useBlinkPlanner().inBatchMode().build()
     val tEnv = TableEnvironment.create(setting)
-    tEnv.sqlUpdate(ddl)
+    tEnv.executeSql(ddl)
 
     val result = Lists.newArrayList(tEnv.sqlQuery(sqlQuery).execute().collect())
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala
index 817940f..14710de 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/AggregateITCase.scala
@@ -1243,14 +1243,13 @@ class AggregateITCase(
       Array[String]("c", "bMax"), Array[TypeInformation[_]](Types.STRING, Types.LONG))
     tEnv.registerTableSink("testSink", tableSink)
 
-    tEnv.sqlUpdate(
+    execInsertSqlAndWaitResult(
       """
         |insert into testSink
         |select c, max(b) from
         | (select b, c, true as f from MyTable) t
         |group by c, f
       """.stripMargin)
-    tEnv.execute("test")
 
     val expected = List("A,1", "B,2", "C,3")
     assertEquals(expected.sorted, tableSink.getUpsertResults.sorted)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
index 1c62821..c73faf5 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CalcITCase.scala
@@ -216,8 +216,7 @@ class CalcITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sqlQuery)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("0,0,0", "1,1,1", "2,2,2")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala
index e60a61c..69dce1f 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/CorrelateITCase.scala
@@ -239,8 +239,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("1,2,,null", "1,3,,null")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -261,8 +260,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("3018-06-10", "2018-06-03", "2018-06-01", "2018-06-02")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -283,8 +281,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("1,3018-06-10", "1,2018-06-03", "1,2018-06-01", "1,2018-06-02")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -304,8 +301,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("a")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -325,8 +321,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     // output two null
     val expected = List("null", "null")
@@ -347,8 +342,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("1,a")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
@@ -368,8 +362,7 @@ class CorrelateITCase extends StreamingTestBase {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("2,null", "3,null")
     assertEquals(expected.sorted, sink.getAppendResults.sorted)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala
index b3c1e9e..747eaa6 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/Limit0RemoveITCase.scala
@@ -38,8 +38,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     assertEquals(0, sink.getAppendResults.size)
   }
@@ -55,8 +54,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     assertEquals(0, sink.getAppendResults.size)
   }
@@ -72,8 +70,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     assertEquals(0, sink.getAppendResults.size)
   }
@@ -93,8 +90,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     assertEquals(0, sink.getAppendResults.size)
   }
@@ -114,8 +110,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = Seq("1", "2", "3", "4", "5", "6")
     assertEquals(expected, sink.getAppendResults.sorted)
@@ -136,8 +131,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingRetractTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     assertEquals(0, sink.getRawResults.size)
   }
@@ -157,8 +151,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingRetractTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = Seq("1", "2", "3", "4", "5", "6")
     assertEquals(expected, sink.getRetractResults.sorted)
@@ -179,8 +172,7 @@ class Limit0RemoveITCase extends StreamingTestBase() {
     val result = tEnv.sqlQuery(sql)
     val sink = TestSinkUtil.configureSink(result, new TestingAppendTableSink())
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     assertEquals(0, sink.getAppendResults.size)
   }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala
index aa6b91e..46b904b 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/RankITCase.scala
@@ -131,8 +131,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "book,4,11,1",
@@ -194,8 +193,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val updatedExpected = List(
       "book,5,800,1",
@@ -257,8 +255,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val updatedExpected = List(
       "book,3,110,1",
@@ -303,8 +300,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val updatedExpected = List(
       "book,1,22,1",
@@ -357,8 +353,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val updatedExpected = List(
       "book,10,1300.0,1",
@@ -402,8 +397,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val updatedExpected = List(
       "book,2,19,2",
@@ -537,8 +531,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "book,1,5,4",
@@ -596,8 +589,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "book,3,2,2",
@@ -656,8 +648,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "(true,1,book,a,1,1)", "(true,2,book,b,1,1)", "(true,3,book,c,1,1)",
@@ -718,8 +709,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "(true,1,book,a,1,1)",
@@ -789,8 +779,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "book,1,5,4",
@@ -853,8 +842,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "(true,book,1,11,1)",
@@ -932,8 +920,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "(true,book,1,100)",
@@ -1017,8 +1004,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "(true,book,1,1)",
@@ -1084,8 +1070,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema.getFieldNames,
         schema.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", table)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table, "MySink")
 
     val expected = List(
       "(true,book,12,1)",
@@ -1213,8 +1198,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema1.getFieldNames, schema1
       .getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink1", sink1)
-    tEnv.insertInto("MySink1", table1)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table1, "MySink1")
 
     val table2 = tEnv.sqlQuery(
       s"""
@@ -1230,8 +1214,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema2.getFieldNames, schema2
       .getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink2", sink2)
-    tEnv.insertInto("MySink2", table2)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table2, "MySink2")
 
     val expected1 = List(
       "book,1,25,1",
@@ -1287,8 +1270,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema1.getFieldNames,
         schema1.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink1", sink1)
-    tEnv.insertInto("MySink1", table1)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table1, "MySink1")
 
     val table2 = tEnv.sqlQuery(
       s"""
@@ -1304,8 +1286,7 @@ class RankITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       configure(schema2.getFieldNames,
         schema2.getFieldDataTypes.map(_.nullable()).map(fromDataTypeToTypeInfo))
     tEnv.registerTableSink("MySink2", sink2)
-    tEnv.insertInto("MySink2", table2)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(table2, "MySink2")
 
     val expected1 = List(
       "book,1,2,1",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala
index 250f5a4..4e03680 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TemporalJoinITCase.scala
@@ -145,13 +145,13 @@ class TemporalJoinITCase(state: StateBackendMode)
     tEnv.createTemporarySystemFunction(
       "Rates",
       tEnv
-        .scan("FilteredRatesHistory")
+        .from("FilteredRatesHistory")
         .createTemporalTableFunction($"rowtime", $"currency"))
     tEnv.registerTable("TemporalJoinResult", tEnv.sqlQuery(sqlQuery))
 
     // Scan from registered table to test for interplay between
     // LogicalCorrelateToTemporalTableJoinRule and TableScanRule
-    val result = tEnv.scan("TemporalJoinResult").toAppendStream[Row]
+    val result = tEnv.from("TemporalJoinResult").toAppendStream[Row]
     val sink = new TestingAppendSink
     result.addSink(sink)
     env.execute()
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala
index b5e5f36..311c682 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/TimeAttributeITCase.scala
@@ -69,7 +69,7 @@ class TimeAttributeITCase extends StreamingTestBase {
         |FROM src
         |GROUP BY TUMBLE(ts, INTERVAL '0.003' SECOND)
       """.stripMargin
-    tEnv.sqlUpdate(ddl)
+    tEnv.executeSql(ddl)
     val sink = new TestingAppendSink()
     tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink)
     env.execute("SQL JOB")
@@ -106,7 +106,7 @@ class TimeAttributeITCase extends StreamingTestBase {
         |FROM src
         |GROUP BY TUMBLE(ts, INTERVAL '0.003' SECOND)
       """.stripMargin
-    tEnv.sqlUpdate(ddl)
+    tEnv.executeSql(ddl)
     val sink = new TestingAppendSink()
     tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink)
     env.execute("SQL JOB")
@@ -143,7 +143,7 @@ class TimeAttributeITCase extends StreamingTestBase {
         |FROM src
         |GROUP BY TUMBLE(rowtime, INTERVAL '0.003' SECOND)
       """.stripMargin
-    tEnv.sqlUpdate(ddl)
+    tEnv.executeSql(ddl)
     val sink = new TestingAppendSink()
     tEnv.sqlQuery(query).toAppendStream[Row].addSink(sink)
     env.execute("SQL JOB")
@@ -177,7 +177,7 @@ class TimeAttributeITCase extends StreamingTestBase {
         |FROM src
         |GROUP BY TUMBLE(col.ts, INTERVAL '0.003' SECOND)
       """.stripMargin
-    tEnv.sqlUpdate(ddl)
+    tEnv.executeSql(ddl)
     expectedException.expect(classOf[ValidationException])
     expectedException.expectMessage(
       "Nested field 'col.ts' as rowtime attribute is not supported right now.")
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala
index 2cd326a..1784dc0 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/UnnestITCase.scala
@@ -265,8 +265,7 @@ class UnnestITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mo
       Array("a", "b", "v"),
       Array(Types.INT, Types.LONG, Types.STRING))
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = List("1,11,10", "1,11,11", "2,22,20", "3,33,30", "3,33,31")
     assertEquals(expected.sorted, sink.getRetractResults.sorted)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala
index 7eb00c8..f817c66 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/sql/WindowAggregateITCase.scala
@@ -231,8 +231,7 @@ class WindowAggregateITCase(mode: StateBackendMode)
 
     val sink = new TestingUpsertTableSink(Array(0, 1)).configure(fieldNames, fieldTypes)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", result)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(result, "MySink")
 
     val expected = Seq(
       "Hi,1970-01-01T00:00,1970-01-01T00:00:00.005,1,1,1,1,1,1,1",
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala
index 146081b..f921242 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/AggregateITCase.scala
@@ -308,8 +308,7 @@ class AggregateITCase(mode: StateBackendMode) extends StreamingWithStateTestBase
       Array[String]("c", "bMax"), Array[TypeInformation[_]](Types.STRING, Types.LONG))
 
     tEnv.registerTableSink("testSink", tableSink)
-    tEnv.insertInto("testSink", t)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(t, "testSink")
 
     val expected = List("A,1", "B,2", "C,3")
     assertEquals(expected.sorted, tableSink.getUpsertResults.sorted)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala
index 99cc2c7..18e1f79 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/JoinITCase.scala
@@ -135,8 +135,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       Array[TypeInformation[_]](Types.INT, Types.LONG, Types.LONG))
 
     tEnv.registerTableSink("upsertSink", sink)
-    tEnv.insertInto("upsertSink", t)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(t, "upsertSink")
 
     val expected = Seq("0,1,1", "1,2,3", "2,1,1", "3,1,1", "4,1,1", "5,2,3", "6,0,1")
     assertEquals(expected.sorted, sink.getUpsertResults.sorted)
@@ -186,8 +185,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
       .join(rightTable, 'a === 'bb && ('a < 4 || 'a > 4))
       .select('a, 'b, 'c, 'd)
     tEnv.registerTableSink("retractSink", sink)
-    tEnv.insertInto("retractSink", t)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(t, "retractSink")
 
     val expected = Seq("1,1,1,1", "1,1,1,1", "1,1,1,1", "1,1,1,1", "2,2,2,2", "3,3,3,3",
                        "5,5,5,5", "5,5,5,5")
@@ -791,8 +789,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 2))
       .configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", t)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(t, "MySink")
 
     val expected = Seq("1,5,1,2")
     assertEquals(expected.sorted, sink.getUpsertResults.sorted)
@@ -872,8 +869,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingRetractTableSink().configure(
       schema.getFieldNames, schema.getFieldTypes)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", t)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(t, "MySink")
 
     val expected = Seq("1,4,1,2", "1,5,1,2")
     assertEquals(expected.sorted, sink.getRetractResults.sorted)
@@ -1060,8 +1056,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 1))
       .configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.registerTableSink("MySink", sink)
-    tEnv.insertInto("MySink", t)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(t, "MySink")
 
     val expected = Seq("0,1,1", "1,2,3", "2,1,1", "3,1,1", "4,1,1", "5,2,3", "6,0,1")
     assertEquals(expected.sorted, sink.getUpsertResults.sorted)
@@ -1343,8 +1338,7 @@ class JoinITCase(mode: StateBackendMode) extends StreamingWithStateTestBase(mode
     val sink = new TestingUpsertTableSink(Array(0, 1, 2))
       .configure(schema.getFieldNames, schema.getFieldTypes)
     tEnv.registerTableSink("sinkTests", sink)
-    tEnv.insertInto("sinkTests", t)
-    tEnv.execute("test")
+    execInsertTableAndWaitResult(t, "sinkTests")
 
     val expected = Seq("4,1,1,1")
     assertEquals(expected, sink.getUpsertResults)
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala
index 9da84ed..0a72906 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/stream/table/TableSinkITCase.scala
@@ -25,13 +25,15 @@ import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.table.api.scala._
 import org.apache.flink.table.api.{DataTypes, TableException, TableSchema, Tumble, Types}
 import org.apache.flink.table.planner.runtime.utils.TestData.{smallTupleData3, tupleData3, tupleData5}
-import org.apache.flink.table.planner.runtime.utils.{TestingAppendTableSink, TestingRetractTableSink, TestingUpsertTableSink}
+import org.apache.flink.table.planner.runtime.utils.{TableEnvUtil, TestingAppendTableSink, TestingRetractTableSink, TestingUpsertTableSink}
 import org.apache.flink.table.planner.utils.{MemoryTableSourceSinkUtil, TableTestUtil}
 import org.apache.flink.table.sinks._
 import org.apache.flink.test.util.{AbstractTestBase, TestBaseUtils}
 import org.apache.flink.types.Row
+
 import org.junit.Assert._
 import org.junit.Test
+
 import java.io.File
 import java.util.TimeZone
 
@@ -43,6 +45,7 @@ class TableSinkITCase extends AbstractTestBase {
   def testStreamTableSink(): Unit = {
 
     val tmpFile = File.createTempFile("flink-table-sink-test", ".tmp")
+    tmpFile.delete()
     tmpFile.deleteOnExit()
     val path = tmpFile.toURI.toString
 
@@ -63,12 +66,10 @@ class TableSinkITCase extends AbstractTestBase {
       .assignAscendingTimestamps(_._2)
       .map(x => x).setParallelism(4) // increase DOP to 4
 
-    input.toTable(tEnv, 'a, 'b.rowtime, 'c)
+    val table = input.toTable(tEnv, 'a, 'b.rowtime, 'c)
       .where('a < 5 || 'a > 17)
       .select(ifThenElse('a < 4, nullOf(Types.INT()), 'a), 'c, 'b)
-      .insertInto("csvSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "csvSink")
 
     val expected = Seq(
       ",Hello world,1970-01-01 00:00:00.002",
@@ -91,8 +92,8 @@ class TableSinkITCase extends AbstractTestBase {
     val tEnv = StreamTableEnvironment.create(env, TableTestUtil.STREAM_SETTING)
 
     val t = env.fromCollection(tupleData3)
-        .assignAscendingTimestamps(_._1.toLong)
-        .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
+      .assignAscendingTimestamps(_._1.toLong)
+      .toTable(tEnv, 'id, 'num, 'text, 'rowtime.rowtime)
 
     val sink = new TestingAppendTableSink(TimeZone.getDefault)
     tEnv.registerTableSink(
@@ -101,12 +102,10 @@ class TableSinkITCase extends AbstractTestBase {
         Array[String]("t", "icnt", "nsum"),
         Array[TypeInformation[_]](Types.SQL_TIMESTAMP, Types.LONG, Types.LONG)))
 
-    t.window(Tumble over 5.millis on 'rowtime as 'w)
+    val table = t.window(Tumble over 5.millis on 'rowtime as 'w)
       .groupBy('w)
       .select('w.end as 't, 'id.count as 'icnt, 'num.sum as 'nsum)
-      .insertInto("appendSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "appendSink")
 
     val result = sink.getAppendResults.sorted
     val expected = List(
@@ -137,9 +136,8 @@ class TableSinkITCase extends AbstractTestBase {
         Array[String]("t", "item"),
         Array[TypeInformation[_]](Types.INT(), Types.ROW(Types.LONG, Types.STRING()))))
 
-    tEnv.sqlUpdate("INSERT INTO appendSink SELECT id, ROW(num, text) FROM src")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertSqlAndWaitResult(
+      tEnv, "INSERT INTO appendSink SELECT id, ROW(num, text) FROM src")
 
     val result = sink.getAppendResults.sorted
     val expected = List(
@@ -166,11 +164,9 @@ class TableSinkITCase extends AbstractTestBase {
         Array[String]("c", "g"),
         Array[TypeInformation[_]](Types.STRING, Types.STRING)))
 
-    ds1.join(ds2).where('b === 'e)
+    val table = ds1.join(ds2).where('b === 'e)
       .select('c, 'g)
-      .insertInto("appendSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "appendSink")
 
     val result = sink.getAppendResults.sorted
     val expected = List("Hi,Hallo", "Hello,Hallo Welt", "Hello world,Hallo Welt").sorted
@@ -195,12 +191,10 @@ class TableSinkITCase extends AbstractTestBase {
         Array[String]("len", "icnt", "nsum"),
         Array[TypeInformation[_]](Types.INT, Types.LONG, Types.DECIMAL())))
 
-    t.select('id, 'num, 'text.charLength() as 'len)
+    val table = t.select('id, 'num, 'text.charLength() as 'len)
       .groupBy('len)
       .select('len, 'id.count as 'icnt, 'num.sum as 'nsum)
-      .insertInto("retractSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "retractSink")
 
     val retracted = sink.getRetractResults.sorted
     val expected = List(
@@ -233,12 +227,10 @@ class TableSinkITCase extends AbstractTestBase {
         Array[String]("t", "icnt", "nsum"),
         Array[TypeInformation[_]](Types.SQL_TIMESTAMP, Types.LONG, Types.LONG)))
 
-    t.window(Tumble over 5.millis on 'rowtime as 'w)
+    val table = t.window(Tumble over 5.millis on 'rowtime as 'w)
       .groupBy('w)
       .select('w.end as 't, 'id.count as 'icnt, 'num.sum as 'nsum)
-      .insertInto("retractSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "retractSink")
 
     assertFalse(
       "Received retraction messages for append only table",
@@ -274,15 +266,13 @@ class TableSinkITCase extends AbstractTestBase {
     sink.expectedIsAppendOnly = Some(false)
     tEnv.registerTableSink("upsertSink", sink)
 
-    t.select('id, 'num, 'text.charLength() as 'len, ('id > 0) as 'cTrue)
+    val table = t.select('id, 'num, 'text.charLength() as 'len, ('id > 0) as 'cTrue)
       .groupBy('len, 'cTrue)
       // test query field name is different with registered sink field name
       .select('len, 'id.count as 'count, 'cTrue)
       .groupBy('count, 'cTrue)
       .select('count, 'len.count as 'lencnt, 'cTrue)
-      .insertInto("upsertSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "upsertSink")
 
     assertTrue(
       "Results must include delete messages",
@@ -315,13 +305,11 @@ class TableSinkITCase extends AbstractTestBase {
     sink.expectedIsAppendOnly = Some(true)
     tEnv.registerTableSink("upsertSink", sink)
 
-    t.window(Tumble over 5.millis on 'rowtime as 'w)
+    val table = t.window(Tumble over 5.millis on 'rowtime as 'w)
       .groupBy('w, 'num)
       // test query field name is different with registered sink field name
       .select('num, 'w.end as 'window_end, 'id.count as 'icnt)
-      .insertInto("upsertSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "upsertSink")
 
     assertFalse(
       "Received retraction messages for append only table",
@@ -363,12 +351,10 @@ class TableSinkITCase extends AbstractTestBase {
         Array[TypeInformation[_]]
           (Types.SQL_TIMESTAMP, Types.SQL_TIMESTAMP, Types.LONG, Types.LONG)))
 
-    t.window(Tumble over 5.millis on 'rowtime as 'w)
+    val table = t.window(Tumble over 5.millis on 'rowtime as 'w)
       .groupBy('w, 'num)
       .select('w.start as 'wstart, 'w.end as 'wend, 'num, 'id.count as 'icnt)
-      .insertInto("upsertSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "upsertSink")
 
     assertFalse(
       "Received retraction messages for append only table",
@@ -408,12 +394,10 @@ class TableSinkITCase extends AbstractTestBase {
         Array[String]("wend", "cnt"),
         Array[TypeInformation[_]](Types.SQL_TIMESTAMP, Types.LONG)))
 
-    t.window(Tumble over 5.millis on 'rowtime as 'w)
+    val table = t.window(Tumble over 5.millis on 'rowtime as 'w)
       .groupBy('w, 'num)
       .select('w.end as 'wend, 'id.count as 'cnt)
-      .insertInto("upsertSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "upsertSink")
 
     assertFalse(
       "Received retraction messages for append only table",
@@ -453,12 +437,10 @@ class TableSinkITCase extends AbstractTestBase {
         Array[String]("num", "cnt"),
         Array[TypeInformation[_]](Types.LONG, Types.LONG)))
 
-    t.window(Tumble over 5.millis on 'rowtime as 'w)
+    val table = t.window(Tumble over 5.millis on 'rowtime as 'w)
       .groupBy('w, 'num)
       .select('num, 'id.count as 'cnt)
-      .insertInto("upsertSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "upsertSink")
 
     assertFalse(
       "Received retraction messages for append only table",
@@ -508,15 +490,13 @@ class TableSinkITCase extends AbstractTestBase {
     //   5, 5
     //   6, 6
 
-    t.groupBy('num)
+    val table = t.groupBy('num)
       .select('num, 'id.count as 'cnt)
       .where('cnt <= 3)
-      .insertInto("upsertSink")
-
-    tEnv.execute("job name")
+    TableEnvUtil.execInsertTableAndWaitResult(table, "upsertSink")
 
     val expectedWithFilter = List("1,1", "2,2", "3,3")
-     assertEquals(expectedWithFilter.sorted, sink.getUpsertResults.sorted)
+    assertEquals(expectedWithFilter.sorted, sink.getUpsertResults.sorted)
   }
 
   @Test(expected = classOf[TableException])
@@ -573,13 +553,11 @@ class TableSinkITCase extends AbstractTestBase {
     MemoryTableSourceSinkUtil.createDataTypeAppendStreamTable(
       tEnv, schema, "testSink")
 
-    env.fromCollection(tupleData3)
-        .toTable(tEnv, 'a, 'b, 'c)
-        .where('a > 20)
-        .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5)))
-        .insertInto("testSink")
-
-    tEnv.execute("")
+    val table = env.fromCollection(tupleData3)
+      .toTable(tEnv, 'a, 'b, 'c)
+      .where('a > 20)
+      .select("12345", 55.cast(DataTypes.DECIMAL(10, 0)), "12345".cast(DataTypes.CHAR(5)))
+    TableEnvUtil.execInsertTableAndWaitResult(table, "testSink")
 
     val results = MemoryTableSourceSinkUtil.tableDataStrings.asJava
     val expected = Seq("12345,55,12345").mkString("\n")
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala
index f65dd04..b7b3cdd 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTableEnvUtil.scala
@@ -55,9 +55,8 @@ object BatchTableEnvUtil {
     sink.init(typeSerializer.asInstanceOf[TypeSerializer[T]], id)
     val sinkName = UUID.randomUUID().toString
     tEnv.registerTableSink(sinkName, sink)
-    tEnv.insertInto(s"`$sinkName`", table)
 
-    val res = tEnv.execute("test")
+    val res = TableEnvUtil.execInsertTableAndWaitResult(table, s"`$sinkName`")
     val accResult: JArrayList[Array[Byte]] = res.getAccumulatorResult(id)
     SerializedListAccumulator.deserializeList(accResult, typeSerializer)
   }
@@ -278,7 +277,7 @@ object BatchTableEnvUtil {
     boundedStream.setParallelism(1)
     val name = if (tableName == null) UUID.randomUUID().toString else tableName
     registerBoundedStreamInternal(tEnv, name, boundedStream, Option(fieldNames), None, statistic)
-    tEnv.scan(name)
+    tEnv.from("`" + name + "`")
   }
 
   /**
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala
index 0ac3083..44c983e 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/BatchTestBase.scala
@@ -18,6 +18,7 @@
 
 package org.apache.flink.table.planner.runtime.utils
 
+import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.java.tuple.Tuple
 import org.apache.flink.streaming.api.datastream.DataStream
@@ -303,7 +304,15 @@ class BatchTestBase extends BatchAbstractTestBase {
     executeQuery(table)
   }
 
-  private def prepareResult(seq: Seq[Row], isSorted: Boolean) = {
+  def execInsertSqlAndWaitResult(insert: String): JobExecutionResult = {
+    TableEnvUtil.execInsertSqlAndWaitResult(tEnv, insert)
+  }
+
+  def execInsertTableAndWaitResult(table: Table, targetPath: String): JobExecutionResult = {
+    TableEnvUtil.execInsertTableAndWaitResult(table, targetPath)
+  }
+
+  private def prepareResult(seq: Seq[Row], isSorted: Boolean): Seq[String] = {
     if (!isSorted) seq.map(_.toString).sortBy(s => s) else seq.map(_.toString)
   }
 
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala
index fa8b732..0951ff1 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/StreamingTestBase.scala
@@ -18,11 +18,14 @@
 
 package org.apache.flink.table.planner.runtime.utils
 
+import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.streaming.api.TimeCharacteristic
 import org.apache.flink.streaming.api.scala.StreamExecutionEnvironment
 import org.apache.flink.table.api.scala.StreamTableEnvironment
 import org.apache.flink.table.api.{EnvironmentSettings, ImplicitExpressionConversions}
 import org.apache.flink.table.planner.factories.TestValuesTableFactory
+import org.apache.flink.table.api.{EnvironmentSettings, Table}
+
 import org.apache.flink.test.util.AbstractTestBase
 import org.apache.flink.types.Row
 
@@ -73,4 +76,12 @@ class StreamingTestBase extends AbstractTestBase {
     }
     row
   }
+
+  def execInsertSqlAndWaitResult(insert: String): JobExecutionResult = {
+    TableEnvUtil.execInsertSqlAndWaitResult(tEnv, insert)
+  }
+
+  def execInsertTableAndWaitResult(table: Table, targetPath: String): JobExecutionResult = {
+    TableEnvUtil.execInsertTableAndWaitResult(table, targetPath)
+  }
 }
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableEnvUtil.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableEnvUtil.scala
new file mode 100644
index 0000000..bac0282
--- /dev/null
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/utils/TableEnvUtil.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.table.planner.runtime.utils
+
+import org.apache.flink.api.common.JobExecutionResult
+import org.apache.flink.table.api.{Table, TableEnvironment}
+
+object TableEnvUtil {
+
+  def execInsertSqlAndWaitResult(tEnv: TableEnvironment, insert: String): JobExecutionResult = {
+    val tableResult = tEnv.executeSql(insert)
+    // wait to finish
+    tableResult.getJobClient.get
+      .getJobExecutionResult(Thread.currentThread.getContextClassLoader)
+      .get
+  }
+
+  def execInsertTableAndWaitResult(table: Table, targetPath: String): JobExecutionResult = {
+    val tableResult =  table.executeInsert(targetPath)
+    // wait to finish
+    tableResult.getJobClient.get
+      .getJobExecutionResult(Thread.currentThread.getContextClassLoader)
+      .get
+  }
+}
diff --git a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
index 2d50f43..3b78afd 100644
--- a/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
+++ b/flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/utils/TableTestBase.scala
@@ -17,12 +17,12 @@
  */
 package org.apache.flink.table.planner.utils
 
-import org.apache.flink.api.common.JobExecutionResult
 import org.apache.flink.api.common.typeinfo.{AtomicType, TypeInformation}
 import org.apache.flink.api.java.typeutils.{PojoTypeInfo, RowTypeInfo, TupleTypeInfo}
 import org.apache.flink.api.scala.typeutils.CaseClassTypeInfo
 import org.apache.flink.streaming.api.datastream.DataStream
 import org.apache.flink.streaming.api.environment.{LocalStreamEnvironment, StreamExecutionEnvironment}
+import org.apache.flink.streaming.api.graph.GlobalDataExchangeMode
 import org.apache.flink.streaming.api.scala.{StreamExecutionEnvironment => ScalaStreamExecEnv}
 import org.apache.flink.streaming.api.{TimeCharacteristic, environment}
 import org.apache.flink.table.api._
@@ -36,14 +36,13 @@ import org.apache.flink.table.catalog.{CatalogManager, FunctionCatalog, GenericI
 import org.apache.flink.table.data.RowData
 import org.apache.flink.table.delegation.{Executor, ExecutorFactory, PlannerFactory}
 import org.apache.flink.table.descriptors.ConnectorDescriptorValidator.CONNECTOR_TYPE
-import org.apache.flink.table.descriptors.{CustomConnectorDescriptor, DescriptorProperties, Schema}
 import org.apache.flink.table.descriptors.Schema.SCHEMA
+import org.apache.flink.table.descriptors.{CustomConnectorDescriptor, DescriptorProperties, Schema}
 import org.apache.flink.table.expressions.Expression
 import org.apache.flink.table.factories.{ComponentFactoryService, StreamTableSourceFactory}
 import org.apache.flink.table.functions._
 import org.apache.flink.table.module.ModuleManager
-import org.apache.flink.table.operations.ddl.CreateTableOperation
-import org.apache.flink.table.operations.{CatalogSinkModifyOperation, ModifyOperation, QueryOperation}
+import org.apache.flink.table.operations.{CatalogSinkModifyOperation, ModifyOperation, Operation, QueryOperation}
 import org.apache.flink.table.planner.calcite.CalciteConfig
 import org.apache.flink.table.planner.delegation.PlannerBase
 import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable
@@ -62,6 +61,7 @@ import org.apache.flink.table.types.logical.LogicalType
 import org.apache.flink.table.types.utils.TypeConversions
 import org.apache.flink.table.typeutils.FieldInfoUtils
 import org.apache.flink.types.Row
+
 import org.apache.calcite.avatica.util.TimeUnit
 import org.apache.calcite.rel.RelNode
 import org.apache.calcite.sql.parser.SqlParserPos
@@ -70,11 +70,10 @@ import org.apache.commons.lang3.SystemUtils
 import org.junit.Assert.{assertEquals, assertTrue}
 import org.junit.Rule
 import org.junit.rules.{ExpectedException, TemporaryFolder, TestName}
+
 import _root_.java.math.{BigDecimal => JBigDecimal}
 import _root_.java.util
 
-import org.apache.flink.streaming.api.graph.GlobalDataExchangeMode
-
 import _root_.scala.collection.JavaConversions._
 import _root_.scala.io.Source
 
@@ -149,17 +148,11 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
     getTableEnv.asInstanceOf[TableEnvironmentImpl].getPlanner.asInstanceOf[PlannerBase]
   }
 
-  def writeToSink(table: Table, sink: TableSink[_], sinkName: String): Unit = {
-    val tableEnv = getTableEnv
-    tableEnv.registerTableSink(sinkName, sink)
-    tableEnv.insertInto(sinkName, table)
-  }
-
   /**
     * Creates a table with the given DDL SQL string.
     */
   def addTable(ddl: String): Unit = {
-    getTableEnv.sqlUpdate(ddl)
+    getTableEnv.executeSql(ddl)
   }
 
   /**
@@ -176,7 +169,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
     val dataStream = env.fromElements[T]().javaStream
     val tableEnv = getTableEnv
     TableTestUtil.createTemporaryView(tableEnv, name, dataStream, Some(fields.toArray))
-    tableEnv.scan(name)
+    tableEnv.from(name)
   }
 
   /**
@@ -250,7 +243,7 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
       name: String,
       tableSource: TableSource[_]): Table = {
     getTableEnv.registerTableSource(name, tableSource)
-    getTableEnv.scan(name)
+    getTableEnv.from(name)
   }
 
   /**
@@ -275,35 +268,27 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
   }
 
   def verifyPlan(sql: String): Unit = {
-    doVerifyPlan(
-      sql,
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withRowType = false,
-      printPlanBefore = true)
+    doVerifyPlan(sql, Array.empty[ExplainDetail], withRowType = false, printPlanBefore = true)
+  }
+
+  def verifyPlan(sql: String, extraDetails: ExplainDetail*): Unit = {
+    doVerifyPlan(sql, extraDetails.toArray, withRowType = false, printPlanBefore = true)
   }
 
   def verifyPlan(table: Table): Unit = {
-    doVerifyPlan(
-      table,
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withRowType = false,
-      printPlanBefore = true)
+    doVerifyPlan(table, Array.empty[ExplainDetail], withRowType = false, printPlanBefore = true)
+  }
+
+  def verifyPlan(table: Table, extraDetails: ExplainDetail*): Unit = {
+    doVerifyPlan(table, extraDetails.toArray, withRowType = false, printPlanBefore = true)
   }
 
   def verifyPlanWithType(sql: String): Unit = {
-    doVerifyPlan(
-      sql,
-      explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withRowType = true,
-      printPlanBefore = true)
+    doVerifyPlan(sql, Array.empty[ExplainDetail], withRowType = true, printPlanBefore = true)
   }
 
   def verifyPlanWithType(table: Table): Unit = {
-    doVerifyPlan(
-      table,
-      explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withRowType = true,
-      printPlanBefore = true)
+    doVerifyPlan(table, Array.empty[ExplainDetail], withRowType = true, printPlanBefore = true)
   }
 
   def verifyPlanNotExpected(sql: String, notExpected: String*): Unit = {
@@ -313,59 +298,54 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
   def verifyPlanNotExpected(table: Table, notExpected: String*): Unit = {
     require(notExpected.nonEmpty)
     val relNode = TableTestUtil.toRelNode(table)
-    val optimizedPlan = getOptimizedPlan(
-      Array(relNode),
-      explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withChangelogTraits = false,
-      withRowType = false)
+    val optimizedPlan = getOptimizedPlan(Array(relNode), Array.empty, withRowType = false)
     val result = notExpected.forall(!optimizedPlan.contains(_))
     val message = s"\nactual plan:\n$optimizedPlan\nnot expected:\n${notExpected.mkString(", ")}"
     assertTrue(message, result)
   }
 
-  def verifyExplain(): Unit = verifyExplain(extended = false)
-
-  def verifyExplain(extended: Boolean): Unit = doVerifyExplain(extended = extended)
+  def verifyExplain(stmtSet: StatementSet, extraDetails: ExplainDetail*): Unit = {
+    doVerifyExplain(
+      stmtSet.explain(extraDetails: _*),
+      extraDetails.contains(ExplainDetail.ESTIMATED_COST))
+  }
 
-  def verifyExplain(sql: String): Unit = verifyExplain(sql, extended = false)
+  def verifyExplain(sql: String): Unit = verifyExplain(getTableEnv.sqlQuery(sql))
 
-  def verifyExplain(sql: String, extended: Boolean): Unit = {
+  def verifyExplain(sql: String, extraDetails: ExplainDetail*): Unit = {
     val table = getTableEnv.sqlQuery(sql)
-    verifyExplain(table, extended)
+    verifyExplain(table, extraDetails: _*)
   }
 
-  def verifyExplain(table: Table): Unit = verifyExplain(table, extended = false)
+  def verifyExplain(table: Table): Unit = {
+    doVerifyExplain(table.explain(), needReplaceEstimatedCost = false)
+  }
 
-  def verifyExplain(table: Table, extended: Boolean): Unit = {
-    doVerifyExplain(Some(table), extended = extended)
+  def verifyExplain(table: Table, extraDetails: ExplainDetail*): Unit = {
+    doVerifyExplain(
+      table.explain(extraDetails: _*),
+      extraDetails.contains(ExplainDetail.ESTIMATED_COST))
   }
 
-  def doVerifyPlan(
-      sql: String,
-      explainLevel: SqlExplainLevel,
-      withRowType: Boolean,
-      printPlanBefore: Boolean): Unit = {
-    doVerifyPlan(
-      sql = sql,
-      explainLevel = explainLevel,
-      withChangelogTraits = false,
-      withRowType = withRowType,
-      printPlanBefore = printPlanBefore)
+  def verifyExplainInsert(
+      table: Table,
+      sink: TableSink[_],
+      targetPath: String,
+      extraDetails: ExplainDetail*): Unit = {
+    val stmtSet = getTableEnv.createStatementSet()
+    getTableEnv.registerTableSink(targetPath, sink)
+    stmtSet.addInsert(targetPath, table)
+    verifyExplain(stmtSet, extraDetails: _*)
   }
 
   def doVerifyPlan(
       sql: String,
-      explainLevel: SqlExplainLevel,
-      withChangelogTraits: Boolean,
+      extraDetails: Array[ExplainDetail],
       withRowType: Boolean,
       printPlanBefore: Boolean): Unit = {
     val table = getTableEnv.sqlQuery(sql)
     val relNode = TableTestUtil.toRelNode(table)
-    val optimizedPlan = getOptimizedPlan(
-      Array(relNode),
-      explainLevel,
-      withChangelogTraits = withChangelogTraits,
-      withRowType = withRowType)
+    val optimizedPlan = getOptimizedPlan(Array(relNode), extraDetails, withRowType = withRowType)
 
     assertEqualsOrExpand("sql", sql)
 
@@ -387,38 +367,35 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
     val table = getTableEnv.sqlQuery(sql)
     doVerifyPlan(
       table,
-      explainLevel = SqlExplainLevel.EXPPLAN_ATTRIBUTES,
+      Array.empty,
       withRowType = false,
-      withChangelogTraits = false,
       printResource = true,
       printPlanBefore = false)
   }
 
   def doVerifyPlan(
       table: Table,
-      explainLevel: SqlExplainLevel,
+      extraDetails: Array[ExplainDetail],
       withRowType: Boolean,
       printPlanBefore: Boolean): Unit = {
     doVerifyPlan(
       table = table,
-      explainLevel = explainLevel,
-      withChangelogTraits = false,
+      extraDetails,
       withRowType = withRowType,
-      printPlanBefore = printPlanBefore)
+      printPlanBefore = printPlanBefore,
+      printResource = false)
   }
 
   def doVerifyPlan(
       table: Table,
-      explainLevel: SqlExplainLevel,
+      extraDetails: Array[ExplainDetail],
       withRowType: Boolean,
-      withChangelogTraits: Boolean,
       printPlanBefore: Boolean,
-      printResource: Boolean = false): Unit = {
+      printResource: Boolean): Unit = {
     val relNode = TableTestUtil.toRelNode(table)
     val optimizedPlan = getOptimizedPlan(
       Array(relNode),
-      explainLevel,
-      withChangelogTraits = withChangelogTraits,
+      extraDetails,
       withRowType = withRowType,
       withResource = printResource)
 
@@ -435,12 +412,8 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
     assertEqualsOrExpand("planAfter", actual.toString, expand = false)
   }
 
-  private def doVerifyExplain(table: Option[Table] = None, extended: Boolean = false): Unit = {
-    val explainResult = table match {
-      case Some(t) => getTableEnv.explain(t, extended)
-      case _ => getTableEnv.explain(extended)
-    }
-    val actual = if (extended) {
+  private def doVerifyExplain(explainResult: String, needReplaceEstimatedCost: Boolean): Unit = {
+    val actual = if (needReplaceEstimatedCost) {
       replaceEstimatedCost(explainResult)
     } else {
       explainResult
@@ -450,13 +423,19 @@ abstract class TableTestUtilBase(test: TableTestBase, isStreamingMode: Boolean)
 
   protected def getOptimizedPlan(
       relNodes: Array[RelNode],
-      explainLevel: SqlExplainLevel,
-      withChangelogTraits: Boolean,
+      extraDetails: Array[ExplainDetail],
       withRowType: Boolean,
       withResource: Boolean = false): String = {
     require(relNodes.nonEmpty)
     val planner = getPlanner
     val optimizedRels = planner.optimize(relNodes)
+    val explainLevel = if (extraDetails.contains(ExplainDetail.ESTIMATED_COST)) {
+      SqlExplainLevel.ALL_ATTRIBUTES
+    } else {
+      SqlExplainLevel.EXPPLAN_ATTRIBUTES
+    }
+    val withChangelogTraits = extraDetails.contains(ExplainDetail.CHANGELOG_MODE)
+
     optimizedRels.head match {
       case _: ExecNode[_, _] =>
         val optimizedNodes = planner.translateToExecNodePlan(optimizedRels)
@@ -576,7 +555,7 @@ abstract class TableTestUtil(
       statistic)
     val table = testingTableEnv.createTable(operation)
     testingTableEnv.registerTable(name, table)
-    testingTableEnv.scan(name)
+    testingTableEnv.from(name)
   }
 
   /**
@@ -602,41 +581,44 @@ abstract class TableTestUtil(
     testingTableEnv.registerFunction(name, function)
   }
 
-  def verifySqlUpdate(sql: String): Unit = {
-    doVerifySqlUpdate(
-      sql,
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withRowType = false,
-      withChangelogTraits = false,
-      printPlanBefore = true)
+  def verifyPlanInsert(sql: String): Unit = {
+    doVerifyPlanInsert(sql, Array.empty, withRowType = false, printPlanBefore = true)
   }
 
-  def verifyPlan(): Unit = {
-    doVerifyPlan(
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withRowType = false,
-      withChangelogTraits = false,
-      printPlanBefore = true)
+  def verifyPlanInsert(
+      table: Table,
+      sink: TableSink[_],
+      targetPath: String,
+      extraDetails: ExplainDetail*): Unit = {
+    val stmtSet = tableEnv.createStatementSet()
+    tableEnv.registerTableSink(targetPath, sink)
+    stmtSet.addInsert(targetPath, table)
+    verifyPlan(stmtSet, extraDetails: _*)
+  }
+
+  def verifyPlan(stmtSet: StatementSet, extraDetails: ExplainDetail*): Unit = {
+    doVerifyPlan(stmtSet, extraDetails.toArray, withRowType = false, printPlanBefore = true)
   }
 
-  def doVerifySqlUpdate(
+  def doVerifyPlanInsert(
       sql: String,
-      explainLevel: SqlExplainLevel,
+      extraDetails: Array[ExplainDetail],
       withRowType: Boolean,
-      withChangelogTraits: Boolean,
       printPlanBefore: Boolean): Unit = {
-    tableEnv.sqlUpdate(sql)
     assertEqualsOrExpand("sql", sql)
-    doVerifyPlan(explainLevel, withRowType, withChangelogTraits, printPlanBefore)
+    val stmtSet = tableEnv.createStatementSet()
+    stmtSet.addInsertSql(sql)
+    doVerifyPlan(stmtSet, extraDetails, withRowType, printPlanBefore)
   }
 
   def doVerifyPlan(
-      explainLevel: SqlExplainLevel,
+      stmtSet: StatementSet,
+      extraDetails: Array[ExplainDetail],
       withRowType: Boolean,
-      withChangelogTraits: Boolean,
       printPlanBefore: Boolean): Unit = {
-    val testTableEnv = tableEnv.asInstanceOf[TestingTableEnvironment]
-    val relNodes = testTableEnv.getBufferedOperations.map(getPlanner.translateToRel)
+    val testStmtSet = stmtSet.asInstanceOf[TestingStatementSet]
+
+    val relNodes = testStmtSet.getOperations.map(getPlanner.translateToRel)
     if (relNodes.isEmpty) {
       throw new TableException("No output table have been created yet. " +
         "A program needs at least one output table that consumes data.\n" +
@@ -645,10 +627,8 @@ abstract class TableTestUtil(
 
     val optimizedPlan = getOptimizedPlan(
       relNodes.toArray,
-      explainLevel,
-      withChangelogTraits = withChangelogTraits,
+      extraDetails,
       withRowType = withRowType)
-    testTableEnv.clearBufferedOperations()
 
     if (printPlanBefore) {
       val planBefore = new StringBuilder
@@ -659,7 +639,11 @@ abstract class TableTestUtil(
       assertEqualsOrExpand("planBefore", planBefore.toString())
     }
 
-    val actual = SystemUtils.LINE_SEPARATOR + optimizedPlan
+    val actual = if (extraDetails.contains(ExplainDetail.ESTIMATED_COST)) {
+      SystemUtils.LINE_SEPARATOR + replaceEstimatedCost(optimizedPlan)
+    } else {
+      SystemUtils.LINE_SEPARATOR + optimizedPlan
+    }
     assertEqualsOrExpand("planAfter", actual.toString, expand = false)
   }
 }
@@ -778,32 +762,6 @@ case class StreamTableTestUtil(
     testingTableEnv.registerTable(tableName, testingTableEnv.createTable(queryOperation))
   }
 
-  def verifyPlanWithTrait(): Unit = {
-    doVerifyPlan(
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withChangelogTraits = true,
-      withRowType = false,
-      printPlanBefore = true)
-  }
-
-  def verifyPlanWithTrait(sql: String): Unit = {
-    doVerifyPlan(
-      sql,
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withChangelogTraits = true,
-      withRowType = false,
-      printPlanBefore = true)
-  }
-
-  def verifyPlanWithTrait(table: Table): Unit = {
-    doVerifyPlan(
-      table,
-      SqlExplainLevel.EXPPLAN_ATTRIBUTES,
-      withChangelogTraits = true,
-      withRowType = false,
-      printPlanBefore = true)
-  }
-
   def buildStreamProgram(firstProgramNameToRemove: String): Unit = {
     val program = FlinkStreamProgram.buildProgram(tableEnv.getConfig.getConfiguration)
     var startRemove = false
@@ -1012,12 +970,6 @@ class TestingTableEnvironment private(
     planner,
     isStreamingMode) {
 
-  private val bufferedOperations: util.List[ModifyOperation] = new util.ArrayList[ModifyOperation]
-
-  def getBufferedOperations: util.List[ModifyOperation] = bufferedOperations
-
-  def clearBufferedOperations(): Unit = bufferedOperations.clear()
-
   // just for testing, remove this method while
   // `<T, ACC> void registerFunction(String name, AggregateFunction<T, ACC> aggregateFunction);`
   // is added into TableEnvironment
@@ -1064,52 +1016,63 @@ class TestingTableEnvironment private(
     )
   }
 
-  override def insertInto(path: String, table: Table): Unit = {
-    val unresolvedIdentifier = parser.parseIdentifier(path)
-    val identifier = catalogManager.qualifyIdentifier(unresolvedIdentifier)
-    buffer(List(new CatalogSinkModifyOperation(identifier, table.getQueryOperation)))
+  override def createTable(tableOperation: QueryOperation): TableImpl = {
+    super.createTable(tableOperation)
   }
 
-  override def sqlUpdate(stmt: String): Unit = {
-    val operations = parser.parse(stmt)
+  override def createStatementSet(): StatementSet = new TestingStatementSet(this)
+}
+
+class TestingStatementSet(tEnv: TestingTableEnvironment) extends StatementSet {
+
+  private val operations: util.List[ModifyOperation] = new util.ArrayList[ModifyOperation]
+
+  def getOperations: util.List[ModifyOperation] = operations
+
+  override def addInsertSql(statement: String): StatementSet = {
+    val operations = tEnv.getParser.parse(statement)
+
     if (operations.size != 1) {
-      throw new TableException(
-        "Unsupported SQL query! sqlUpdate() only accepts a single SQL statement of type INSERT.")
+      throw new TableException("Only single statement is supported.")
     }
-    val operation = operations.get(0)
-    operation match {
-      case modifyOperation: ModifyOperation =>
-        buffer(List(modifyOperation))
-      case createOperation: CreateTableOperation =>
-        catalogManager.createTable(
-          createOperation.getCatalogTable,
-          createOperation.getTableIdentifier,
-          createOperation.isIgnoreIfExists)
-      case _ => throw new TableException(
-        "Unsupported SQL query! sqlUpdate() only accepts a single SQL statements of type INSERT.")
+
+    operations.get(0) match {
+      case op: ModifyOperation =>
+        this.operations.add(op)
+      case _ =>
+        throw new TableException("Only insert statement is supported now.")
     }
+    this
   }
 
-  override def explain(extended: Boolean): String = {
-    planner.explain(bufferedOperations.toList, getExplainDetails(extended): _*)
+  override def addInsert(targetPath: String, table: Table): StatementSet = {
+    this.addInsert(targetPath, table, overwrite = false)
   }
 
-  @throws[Exception]
-  override def execute(jobName: String): JobExecutionResult = {
-    val transformations = planner.translate(bufferedOperations)
-    bufferedOperations.clear()
-    val pipeline = executor.createPipeline(transformations, tableConfig, jobName)
-    execEnv.execute(pipeline)
-  }
+  override def addInsert(targetPath: String, table: Table, overwrite: Boolean): StatementSet = {
+    val unresolvedIdentifier = tEnv.getParser.parseIdentifier(targetPath)
+    val objectIdentifier = tEnv.getCatalogManager.qualifyIdentifier(unresolvedIdentifier)
 
-  override def createTable(tableOperation: QueryOperation): TableImpl = {
-    super.createTable(tableOperation)
+    operations.add(new CatalogSinkModifyOperation(
+      objectIdentifier,
+      table.getQueryOperation,
+      util.Collections.emptyMap[String, String],
+      overwrite,
+      util.Collections.emptyMap[String, String]))
+    this
   }
 
-  private def buffer(modifyOperations: List[ModifyOperation]): Unit = {
-    bufferedOperations.addAll(modifyOperations)
+  override def explain(extraDetails: ExplainDetail*): String = {
+    tEnv.explainInternal(operations.map(o => o.asInstanceOf[Operation]), extraDetails: _*)
   }
 
+  override def execute(): TableResult = {
+    try {
+      tEnv.executeInternal(operations)
+    } finally {
+      operations.clear()
+    }
+  }
 }
 
 object TestingTableEnvironment {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/BatchSelectTableSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/BatchSelectTableSink.java
index fc9bf54..5b360b3 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/BatchSelectTableSink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/BatchSelectTableSink.java
@@ -50,7 +50,8 @@ public class BatchSelectTableSink implements BatchTableSink<Row>, SelectTableSin
 	private JobClient jobClient;
 
 	public BatchSelectTableSink(TableSchema tableSchema) {
-		this.tableSchema = SelectTableSinkSchemaConverter.convert(tableSchema);
+		this.tableSchema =
+				SelectTableSinkSchemaConverter.convertTimeAttributeToRegularTimestamp(tableSchema);
 		this.accumulatorName = new AbstractID().toString();
 		this.typeSerializer = this.tableSchema.toRowType().createSerializer(new ExecutionConfig());
 	}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/SelectTableSinkSchemaConverter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/SelectTableSinkSchemaConverter.java
index 5c8e17c..a52009a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/SelectTableSinkSchemaConverter.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/SelectTableSinkSchemaConverter.java
@@ -25,7 +25,7 @@ import org.apache.flink.table.types.logical.TimestampKind;
 import org.apache.flink.table.types.logical.TimestampType;
 
 /**
- * An utility class that convert time attributes (proc time / event time) to regular timestamps.
+ * An utility class that provides abilities to change {@link TableSchema}.
  */
 class SelectTableSinkSchemaConverter {
 
@@ -33,7 +33,7 @@ class SelectTableSinkSchemaConverter {
 	 * Convert time attributes (proc time / event time) to normal timestamps,
 	 * and return a new {@link TableSchema}.
 	 */
-	static TableSchema convert(TableSchema tableSchema) {
+	static TableSchema convertTimeAttributeToRegularTimestamp(TableSchema tableSchema) {
 		DataType[] oldTypes = tableSchema.getFieldDataTypes();
 		String[] oldNames = tableSchema.getFieldNames();
 
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/StreamSelectTableSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/StreamSelectTableSink.java
index acbcd62..f0a70fe 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/StreamSelectTableSink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/sinks/StreamSelectTableSink.java
@@ -47,7 +47,7 @@ public class StreamSelectTableSink implements AppendStreamTableSink<Row>, Select
 	private final SocketStreamIterator<Row> iterator;
 
 	public StreamSelectTableSink(TableSchema tableSchema) {
-		this.tableSchema = SelectTableSinkSchemaConverter.convert(tableSchema);
+		this.tableSchema = SelectTableSinkSchemaConverter.convertTimeAttributeToRegularTimestamp(tableSchema);
 		this.typeSerializer = this.tableSchema.toRowType().createSerializer(new ExecutionConfig());
 		try {
 			// socket server should be started before running the job
diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala
index 46fe90e..2d75534 100644
--- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala
+++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/api/internal/TableEnvImpl.scala
@@ -594,7 +594,7 @@ abstract class TableEnvImpl(
     }
 
     val sinkIdentifierNames = extractSinkIdentifierNames(operations)
-    val jobName = "insert_into_" + String.join(",", sinkIdentifierNames)
+    val jobName = "insert-into_" + String.join(",", sinkIdentifierNames)
     try {
       val jobClient = execute(dataSinks, jobName)
       val builder = TableSchema.builder()
@@ -903,14 +903,30 @@ abstract class TableEnvImpl(
 
   /**
     * extract sink identifier names from [[ModifyOperation]]s.
+    *
+    * <p>If there are multiple ModifyOperations have same name,
+    * an index suffix will be added at the end of the name to ensure each name is unique.
     */
   private def extractSinkIdentifierNames(operations: JList[ModifyOperation]): JList[String] = {
-    operations.map {
+    val tableNameToCount = new JHashMap[String, Int]()
+    val tableNames = operations.map {
       case catalogSinkModifyOperation: CatalogSinkModifyOperation =>
-        catalogSinkModifyOperation.getTableIdentifier.asSummaryString()
+        val fullName = catalogSinkModifyOperation.getTableIdentifier.asSummaryString()
+        tableNameToCount.put(fullName, tableNameToCount.getOrDefault(fullName, 0) + 1)
+        fullName
       case o =>
         throw new UnsupportedOperationException("Unsupported operation: " + o)
     }
+    val tableNameToIndex = new JHashMap[String, Int]()
+    tableNames.map { tableName =>
+      if (tableNameToCount.get(tableName) == 1) {
+        tableName
+      } else {
+        val index = tableNameToIndex.getOrDefault(tableName, 0) + 1
+        tableNameToIndex.put(tableName, index)
+        tableName + "_" + index
+      }
+    }
   }
 
   /**
diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala
index 0df35fa..bfbd7f4 100644
--- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala
+++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/runtime/batch/sql/TableEnvironmentITCase.scala
@@ -520,6 +520,37 @@ class TableEnvironmentITCase(
   }
 
   @Test
+  def testStatementSetWithSameSinkTableNames(): Unit = {
+    val env = ExecutionEnvironment.getExecutionEnvironment
+    val tEnv = BatchTableEnvironment.create(env)
+    MemoryTableSourceSinkUtil.clear()
+
+    val t = CollectionDataSets.getSmall3TupleDataSet(env).toTable(tEnv).as('a, 'b, 'c)
+    tEnv.registerTable("MyTable", t)
+
+    val sinkPath = _tempFolder.newFile().getAbsolutePath
+    val configuredSink = new TestingOverwritableTableSink(sinkPath)
+      .configure(Array("d", "e", "f"), Array(INT, LONG, STRING))
+    tEnv.registerTableSink("MySink", configuredSink)
+    assertTrue(FileUtils.readFileUtf8(new File(sinkPath)).isEmpty)
+
+    val stmtSet = tEnv.createStatementSet()
+    stmtSet.addInsert("MySink", tEnv.sqlQuery("select * from MyTable where a > 2"), true)
+      .addInsertSql("INSERT OVERWRITE MySink SELECT a, b, c FROM MyTable where a <= 2")
+
+    val tableResult = stmtSet.execute()
+    // wait job finished
+    tableResult.getJobClient.get()
+      .getJobExecutionResult(Thread.currentThread().getContextClassLoader)
+      .get()
+    // only check the schema
+    checkInsertTableResult(
+      tableResult,
+      "default_catalog.default_database.MySink_1",
+      "default_catalog.default_database.MySink_2")
+  }
+
+  @Test
   def testExecuteSelect(): Unit = {
     val env = ExecutionEnvironment.getExecutionEnvironment
     val tEnv = BatchTableEnvironment.create(env)